xintongsong commented on a change in pull request #13864: URL: https://github.com/apache/flink/pull/13864#discussion_r518482982
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/persistence/IntegerResourceVersion.java ########## @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.persistence; + +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; + +/** + * {@link ResourceVersion} implementation with {@link Integer} value. The resource version in + * ZooKeeper is {@link Integer}. + */ +public class IntegerResourceVersion implements ResourceVersion, Comparable<IntegerResourceVersion> { + + private static final IntegerResourceVersion NOT_EXIST = new IntegerResourceVersion(-1); + + private final int value; + + private IntegerResourceVersion(int value) { + this.value = value; + } + + @Override + public int compareTo(@Nonnull IntegerResourceVersion other) { + return Integer.compare(value, other.getValue()); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } else if (obj != null && obj.getClass() == IntegerResourceVersion.class) { + final IntegerResourceVersion that = (IntegerResourceVersion) obj; + return this.value == that.getValue(); + } else { + return false; + } + } + + @Override + public int hashCode() { + return Integer.hashCode(value); + } + + @Override + public boolean isNotExist() { + return this == NOT_EXIST; + } + + @Override + public String toString() { + return "IntegerResourceVersion{" + "value='" + value + '\'' + '}'; + } + + public int getValue() { + return this.value; + } + + public static IntegerResourceVersion notExist() { + return NOT_EXIST; + } + + public static IntegerResourceVersion valueOf(int value) { + Preconditions.checkArgument(value != NOT_EXIST.getValue()); Review comment: I think we should check `value >= 0`. Currently it looks like we can accept any negative value except for `-1`. Might also mention this contract in the JavaDoc of this class. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/persistence/IntegerResourceVersion.java ########## @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.persistence; + +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; + +/** + * {@link ResourceVersion} implementation with {@link Integer} value. The resource version in + * ZooKeeper is {@link Integer}. + */ +public class IntegerResourceVersion implements ResourceVersion, Comparable<IntegerResourceVersion> { Review comment: IDE complains about missing `serialVersionUID`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StringResourceVersion.java ########## @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.persistence; + +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nonnull; + +import java.util.Objects; + +/** + * {@link ResourceVersion} implementation with {@link String} value. The resource version in Kubernetes + * is {@link String}. And they have same length, we could easily compare string. + */ +public class StringResourceVersion implements ResourceVersion, Comparable<StringResourceVersion> { Review comment: IDE complains about missing `serialVersionUID`. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreUtil.java ########## @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmanager; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.util.ZooKeeperUtils; + +/** + * {@link JobGraphStoreUtil} implementation for ZooKeeper. + * + */ +public class ZooKeeperJobGraphStoreUtil implements JobGraphStoreUtil { Review comment: The class can be singleton. ########## File path: flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesJobGraphStoreUtil.java ########## @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.kubernetes.highavailability; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.runtime.jobmanager.JobGraphStoreUtil; + +import static org.apache.flink.kubernetes.utils.Constants.JOB_GRAPH_STORE_KEY_PREFIX; + +/** + * {@link JobGraphStoreUtil} implementation for Kubernetes. + * + */ +public class KubernetesJobGraphStoreUtil implements JobGraphStoreUtil { Review comment: The class can be singleton. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/persistence/ResourceVersion.java ########## @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.persistence; + +import java.io.Serializable; + +/** + * Resource version for specific state handle on the underlying storage. The implementation also needs to implement the + * {@link Comparable} interface so that we could compare the resource versions. Review comment: +1 ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org