XComp commented on a change in pull request #18692:
URL: https://github.com/apache/flink/pull/18692#discussion_r806086781
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java
##########
@@ -52,7 +52,18 @@
*/
public class FileSystemJobResultStore extends AbstractThreadsafeJobResultStore
{
- private static final String DIRTY_SUFFIX = "_DIRTY";
+ @VisibleForTesting static final String FILE_EXTENSION = ".json";
+ @VisibleForTesting static final String DIRTY_FILE_EXTENSION = "_DIRTY" +
FILE_EXTENSION;
+
+ @VisibleForTesting
+ public static boolean hasValidDirtyJobResultStoreEntryExtension(String
filename) {
+ return filename.endsWith(DIRTY_FILE_EXTENSION);
+ }
+
+ @VisibleForTesting
+ public static boolean hasValidJobResultStoreEntryExtension(String
filename) {
+ return filename.endsWith(FILE_EXTENSION);
Review comment:
The constants I use in the `FileSystemJobResultStoreTest` as well. But I
kept them `package-private`. The `hasValid*` can be used in other places like
the `HAJobRunOnMinioS3StoreITCase` and are, therefore, `public`
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaJobRunITCase.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.plugin.PluginManager;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nullable;
+
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * {@code AbstractHaJobRunITCase} runs a job storing in HA mode and provides
{@code abstract}
+ * methods for initializing a specific {@link FileSystem}.
+ */
+@ExtendWith(TestLoggerExtension.class)
+public abstract class AbstractHaJobRunITCase {
+
+ @RegisterExtension
+ private static final AllCallbackWrapper<ZooKeeperExtension>
ZOOKEEPER_EXTENSION =
+ new AllCallbackWrapper<>(new ZooKeeperExtension());
+
+ @RegisterExtension
+ public final EachCallbackWrapper<MiniClusterExtension>
miniClusterExtension =
+ new EachCallbackWrapper<>(
+ new MiniClusterExtension(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberTaskManagers(1)
+ .setNumberSlotsPerTaskManager(1)
+
.setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+ .setConfiguration(getFlinkConfiguration())
+ .build()));
+
+ private Configuration getFlinkConfiguration() {
+ Configuration config = new Configuration();
+ config.set(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+ config.set(
+ HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM,
+ ZOOKEEPER_EXTENSION.getCustomExtension().getConnectString());
+
+ config.set(HighAvailabilityOptions.HA_STORAGE_PATH,
createHAStoragePath());
+
+ updateConfiguration(config);
+
+ FileSystem.initialize(config, loadPluginManager());
+ initializeStorageBackend(config);
+
+ return config;
+ }
+
+ /**
+ * Should return the path to the HA storage which will be injected into
the Flink configuration.
+ *
+ * @see HighAvailabilityOptions#HA_STORAGE_PATH
+ */
+ protected abstract String createHAStoragePath();
+
+ /**
+ * Updates the passed {@link Configuration} to point to the {@link
FileSystem} that's subject to
+ * test.
+ */
+ protected abstract void updateConfiguration(Configuration config);
+
+ /** Runs any additional initialization that are necessary before running
the actual test. */
+ protected void initializeStorageBackend(Configuration config) {}
+
+ @Nullable
+ protected PluginManager loadPluginManager() {
+ // by default, no PluginManager is loaded
+ return null;
+ }
+
+ @Test
+ public void testJobExecutionInHaMode() throws Exception {
+ final MiniCluster flinkCluster =
miniClusterExtension.getCustomExtension().getMiniCluster();
+
+ final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph();
+
+ flinkCluster.runDetached(jobGraph);
Review comment:
The issue is that MiniCluster is running in a separate thread. It fails
while I'm submitting the Job which keeps the job submission to never complete.
But using the non-detached job submission is a good point: I added a timeout to
that to make the test fail earlier. That way, the test would run forever if
there was an initialization error.
Initially, my idea was that I could wait for the `FatalErrorHandler` (if it
would be exposed outside of the MiniCluster) to report an error and fail in
that case. Waiting for a reasonable long time with no error occurring would
have been an indication that the initialization of the MiniCluster succeeded
and I could have continued with the test.
But that approach has the downside that the test has to wait in cases where
it actually succeeds which is bad in terms of test runtime. Making the job
submission fail after a specific timeout sounds like the better approach.
##########
File path:
flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaJobRunITCase.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.api.common.JobStatus;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.plugin.PluginManager;
+import org.apache.flink.core.testutils.AllCallbackWrapper;
+import org.apache.flink.core.testutils.EachCallbackWrapper;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterExtension;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.runtime.zookeeper.ZooKeeperExtension;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.util.TestLoggerExtension;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+import javax.annotation.Nullable;
+
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * {@code AbstractHaJobRunITCase} runs a job storing in HA mode and provides
{@code abstract}
+ * methods for initializing a specific {@link FileSystem}.
+ */
+@ExtendWith(TestLoggerExtension.class)
+public abstract class AbstractHaJobRunITCase {
+
+ @RegisterExtension
+ private static final AllCallbackWrapper<ZooKeeperExtension>
ZOOKEEPER_EXTENSION =
+ new AllCallbackWrapper<>(new ZooKeeperExtension());
+
+ @RegisterExtension
+ public final EachCallbackWrapper<MiniClusterExtension>
miniClusterExtension =
+ new EachCallbackWrapper<>(
+ new MiniClusterExtension(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberTaskManagers(1)
+ .setNumberSlotsPerTaskManager(1)
+
.setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+ .setConfiguration(getFlinkConfiguration())
+ .build()));
+
+ private Configuration getFlinkConfiguration() {
+ Configuration config = new Configuration();
+ config.set(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER");
+ config.set(
+ HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM,
+ ZOOKEEPER_EXTENSION.getCustomExtension().getConnectString());
+
+ config.set(HighAvailabilityOptions.HA_STORAGE_PATH,
createHAStoragePath());
+
+ updateConfiguration(config);
+
+ FileSystem.initialize(config, loadPluginManager());
+ initializeStorageBackend(config);
+
+ return config;
+ }
+
+ /**
+ * Should return the path to the HA storage which will be injected into
the Flink configuration.
+ *
+ * @see HighAvailabilityOptions#HA_STORAGE_PATH
+ */
+ protected abstract String createHAStoragePath();
+
+ /**
+ * Updates the passed {@link Configuration} to point to the {@link
FileSystem} that's subject to
+ * test.
+ */
+ protected abstract void updateConfiguration(Configuration config);
+
+ /** Runs any additional initialization that are necessary before running
the actual test. */
+ protected void initializeStorageBackend(Configuration config) {}
+
+ @Nullable
+ protected PluginManager loadPluginManager() {
+ // by default, no PluginManager is loaded
+ return null;
+ }
+
+ @Test
+ public void testJobExecutionInHaMode() throws Exception {
+ final MiniCluster flinkCluster =
miniClusterExtension.getCustomExtension().getMiniCluster();
+
+ final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph();
+
+ flinkCluster.runDetached(jobGraph);
Review comment:
The issue is that MiniCluster is running in a separate thread. It fails
while I'm submitting the Job which keeps the job submission to never complete.
But using the non-detached job submission is a good point: I added a timeout to
that to make the test fail earlier. That way, the test would run forever if
there was an initialization error.
Initially, my idea was that I could wait for the `FatalErrorHandler` (if it
would be exposed outside of the MiniCluster) to report an error and fail in
that case. Waiting for a reasonable long time with no error occurring would
have been an indication that the initialization of the MiniCluster succeeded
and I could have continued with the test.
But that approach has the downside that the test has to wait in cases where
it actually succeeds which is bad in terms of test runtime. Making the job
submission fail after a specific timeout sounds like the better approach. I
adapted the test accordingly.
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java
##########
@@ -155,18 +155,15 @@ public boolean hasCleanJobResultEntryInternal(JobID
jobId) throws IOException {
@Override
public Set<JobResult> getDirtyResultsInternal() throws IOException {
final Set<JobResult> dirtyResults = new HashSet<>();
- final FileStatus fs = fileSystem.getFileStatus(this.basePath);
- if (fs.isDir()) {
- FileStatus[] statuses = fileSystem.listStatus(this.basePath);
Review comment:
The presto test still fails with this change being reverted.
`PrestoS3FileSystem` does not support `getFileStatus` on empty directories.
Removing the `isDir` check fixes that issue. I do another round over the presto
implementation tomorrow to understand why it is. FLINK-26061 covers the
difference between presto and hadoop s3 fs
*updated the comment to make more sense out of it*
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java
##########
@@ -155,18 +155,15 @@ public boolean hasCleanJobResultEntryInternal(JobID
jobId) throws IOException {
@Override
public Set<JobResult> getDirtyResultsInternal() throws IOException {
final Set<JobResult> dirtyResults = new HashSet<>();
- final FileStatus fs = fileSystem.getFileStatus(this.basePath);
- if (fs.isDir()) {
- FileStatus[] statuses = fileSystem.listStatus(this.basePath);
Review comment:
To clarify: I looked through the code once more. The
`PrestoS3FileSystem` does not support `getFileStatus` on empty directories.
You're right when you said that the `mkdir` call doesn't create anything (see
[PrestoS3FileSystem:520](https://github.com/prestodb/presto/blob/master/presto-hive/src/main/java/com/facebook/presto/hive/s3/PrestoS3FileSystem.java#L520)).
But the `getFileStatus` method tries to get the `FileStatus` of the object at
the given path. If that does not exist, it will look for objects having the
path as a prefix (through `listObject`). A `FileNotFoundException` is thrown if
no objects live underneath the passed path (which corresponds to an empty
directory, see
[PrestoS3FileSystem:361](https://github.com/prestodb/presto/blob/master/presto-hive/src/main/java/com/facebook/presto/hive/s3/PrestoS3FileSystem.java#L361)).
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]