From 9f0295aa3c02e4870b248241cb9094d14863a686 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 26 Feb 2018 18:03:14 +0100 Subject: [PATCH 001/268] [hotfix] Improved logging for task local recovery (cherry picked from commit 56c7560) --- .../PrioritizedOperatorSubtaskState.java | 31 ++++----- .../TaskExecutorLocalStateStoresManager.java | 68 +++++++++++++------ .../state/TaskLocalStateStoreImpl.java | 63 +++++++++++------ .../runtime/state/TaskStateManagerImpl.java | 27 +++++--- .../PrioritizedOperatorSubtaskStateTest.java | 4 +- .../state/TaskStateManagerImplTest.java | 4 +- .../operators/BackendRestorerProcedure.java | 61 +++++++++++------ .../StreamTaskStateInitializerImpl.java | 20 ++++-- .../BackendRestorerProcedureTest.java | 18 ++--- 9 files changed, 193 insertions(+), 103 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java index f48d3110c53c0..512f912620bc5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskState.java @@ -27,7 +27,6 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.function.BiFunction; @@ -79,39 +78,39 @@ public class PrioritizedOperatorSubtaskState { // ----------------------------------------------------------------------------------------------------------------- /** - * Returns an iterator over all alternative snapshots to restore the managed operator state, in the order in which - * we should attempt to restore. + * Returns an immutable list with all alternative snapshots to restore the managed operator state, in the order in + * which we should attempt to restore. */ @Nonnull - public Iterator> getPrioritizedManagedOperatorState() { - return prioritizedManagedOperatorState.iterator(); + public List> getPrioritizedManagedOperatorState() { + return prioritizedManagedOperatorState; } /** - * Returns an iterator over all alternative snapshots to restore the raw operator state, in the order in which we - * should attempt to restore. + * Returns an immutable list with all alternative snapshots to restore the raw operator state, in the order in + * which we should attempt to restore. */ @Nonnull - public Iterator> getPrioritizedRawOperatorState() { - return prioritizedRawOperatorState.iterator(); + public List> getPrioritizedRawOperatorState() { + return prioritizedRawOperatorState; } /** - * Returns an iterator over all alternative snapshots to restore the managed keyed state, in the order in which we - * should attempt to restore. + * Returns an immutable list with all alternative snapshots to restore the managed keyed state, in the order in + * which we should attempt to restore. */ @Nonnull - public Iterator> getPrioritizedManagedKeyedState() { - return prioritizedManagedKeyedState.iterator(); + public List> getPrioritizedManagedKeyedState() { + return prioritizedManagedKeyedState; } /** - * Returns an iterator over all alternative snapshots to restore the raw keyed state, in the order in which we + * Returns an immutable list with all alternative snapshots to restore the raw keyed state, in the order in which we * should attempt to restore. */ @Nonnull - public Iterator> getPrioritizedRawKeyedState() { - return prioritizedRawKeyedState.iterator(); + public List> getPrioritizedRawKeyedState() { + return prioritizedRawKeyedState; } // ----------------------------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java index a940aefcc7f6d..e7a7d8fe35c78 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java @@ -110,38 +110,68 @@ public TaskLocalStateStore localStateStoreForSubtask( "register a new TaskLocalStateStore."); } - final Map taskStateManagers = - this.taskStateStoresByAllocationID.computeIfAbsent(allocationID, k -> new HashMap<>()); + Map taskStateManagers = + this.taskStateStoresByAllocationID.get(allocationID); + + if (taskStateManagers == null) { + taskStateManagers = new HashMap<>(); + this.taskStateStoresByAllocationID.put(allocationID, taskStateManagers); + + if (LOG.isDebugEnabled()) { + LOG.debug("Registered new allocation id {} for local state stores for job {}.", + allocationID, jobId); + } + } final JobVertexSubtaskKey taskKey = new JobVertexSubtaskKey(jobVertexID, subtaskIndex); - // create the allocation base dirs, one inside each root dir. - File[] allocationBaseDirectories = allocationBaseDirectories(allocationID); + TaskLocalStateStoreImpl taskLocalStateStore = taskStateManagers.get(taskKey); - LocalRecoveryDirectoryProviderImpl directoryProvider = new LocalRecoveryDirectoryProviderImpl( - allocationBaseDirectories, - jobId, - jobVertexID, - subtaskIndex); + if (taskLocalStateStore == null) { + + // create the allocation base dirs, one inside each root dir. + File[] allocationBaseDirectories = allocationBaseDirectories(allocationID); + + LocalRecoveryDirectoryProviderImpl directoryProvider = new LocalRecoveryDirectoryProviderImpl( + allocationBaseDirectories, + jobId, + jobVertexID, + subtaskIndex); - LocalRecoveryConfig localRecoveryConfig = new LocalRecoveryConfig( - localRecoveryMode, - directoryProvider); + LocalRecoveryConfig localRecoveryConfig = + new LocalRecoveryConfig(localRecoveryMode, directoryProvider); - return taskStateManagers.computeIfAbsent( - taskKey, - k -> new TaskLocalStateStoreImpl( + taskLocalStateStore = new TaskLocalStateStoreImpl( jobId, allocationID, jobVertexID, subtaskIndex, localRecoveryConfig, - discardExecutor)); + discardExecutor); + + taskStateManagers.put(taskKey, taskLocalStateStore); + + if (LOG.isTraceEnabled()) { + LOG.trace("Registered new local state store with configuration {} for {} - {} - {} under allocation id {}.", + localRecoveryConfig, jobId, jobVertexID, subtaskIndex, allocationID); + } + } else { + if (LOG.isTraceEnabled()) { + LOG.trace("Found existing local state store for {} - {} - {} under allocation id {}.", + jobId, jobVertexID, subtaskIndex, allocationID); + } + } + + return taskLocalStateStore; } } public void releaseLocalStateForAllocationId(@Nonnull AllocationID allocationID) { + if (LOG.isDebugEnabled()) { + LOG.debug("Releasing local state under allocation id {}.", allocationID); + } + Map cleanupLocalStores; synchronized (lock) { @@ -175,7 +205,7 @@ public void shutdown() { ShutdownHookUtil.removeShutdownHook(shutdownHook, getClass().getSimpleName(), LOG); - LOG.debug("Shutting down TaskExecutorLocalStateStoresManager."); + LOG.info("Shutting down TaskExecutorLocalStateStoresManager."); for (Map.Entry> entry : toRelease.entrySet()) { @@ -217,7 +247,7 @@ private void doRelease(Iterable toRelease) { try { stateStore.dispose(); } catch (Exception disposeEx) { - LOG.warn("Exception while disposing local state store " + stateStore, disposeEx); + LOG.warn("Exception while disposing local state store {}.", stateStore, disposeEx); } } } @@ -233,7 +263,7 @@ private void cleanupAllocationBaseDirs(AllocationID allocationID) { try { FileUtils.deleteFileOrDirectory(directory); } catch (IOException e) { - LOG.warn("Exception while deleting local state directory for allocation " + allocationID, e); + LOG.warn("Exception while deleting local state directory for allocation id {}.", allocationID, e); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java index 191c1096aa4d5..bb4f0116dffe5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java @@ -103,15 +103,15 @@ public TaskLocalStateStoreImpl( @Nonnull LocalRecoveryConfig localRecoveryConfig, @Nonnull Executor discardExecutor) { + this.lock = new Object(); + this.storedTaskStateByCheckpointID = new TreeMap<>(); this.jobID = jobID; this.allocationID = allocationID; this.jobVertexID = jobVertexID; this.subtaskIndex = subtaskIndex; this.discardExecutor = discardExecutor; - this.lock = new Object(); - this.storedTaskStateByCheckpointID = new TreeMap<>(); - this.disposed = false; this.localRecoveryConfig = localRecoveryConfig; + this.disposed = false; } @Override @@ -123,8 +123,15 @@ public void storeLocalState( localState = NULL_DUMMY; } - LOG.info("Storing local state for checkpoint {}.", checkpointId); - LOG.debug("Local state for checkpoint {} is {}.", checkpointId, localState); + if (LOG.isTraceEnabled()) { + LOG.debug( + "Stored local state for checkpoint {} in subtask ({} - {} - {}) : {}.", + checkpointId, jobID, jobVertexID, subtaskIndex, localState); + } else if (LOG.isDebugEnabled()) { + LOG.debug( + "Stored local state for checkpoint {} in subtask ({} - {} - {})", + checkpointId, jobID, jobVertexID, subtaskIndex); + } Map toDiscard = new HashMap<>(16); @@ -148,10 +155,21 @@ public void storeLocalState( @Override @Nullable public TaskStateSnapshot retrieveLocalState(long checkpointID) { + + TaskStateSnapshot snapshot; synchronized (lock) { - TaskStateSnapshot snapshot = storedTaskStateByCheckpointID.get(checkpointID); - return snapshot != NULL_DUMMY ? snapshot : null; + snapshot = storedTaskStateByCheckpointID.get(checkpointID); } + + if (LOG.isTraceEnabled()) { + LOG.trace("Found entry for local state for checkpoint {} in subtask ({} - {} - {}) : {}", + checkpointID, jobID, jobVertexID, subtaskIndex, snapshot); + } else if (LOG.isDebugEnabled()) { + LOG.debug("Found entry for local state for checkpoint {} in subtask ({} - {} - {})", + checkpointID, jobID, jobVertexID, subtaskIndex); + } + + return snapshot != NULL_DUMMY ? snapshot : null; } @Override @@ -163,7 +181,8 @@ public LocalRecoveryConfig getLocalRecoveryConfig() { @Override public void confirmCheckpoint(long confirmedCheckpointId) { - LOG.debug("Received confirmation for checkpoint {}. Starting to prune history.", confirmedCheckpointId); + LOG.debug("Received confirmation for checkpoint {} in subtask ({} - {} - {}). Starting to prune history.", + confirmedCheckpointId, jobID, jobVertexID, subtaskIndex); final List> toRemove = new ArrayList<>(); @@ -216,7 +235,8 @@ public CompletableFuture dispose() { try { deleteDirectory(subtaskBaseDirectory); } catch (IOException e) { - LOG.warn("Exception when deleting local recovery subtask base dir: " + subtaskBaseDirectory, e); + LOG.warn("Exception when deleting local recovery subtask base directory {} in subtask ({} - {} - {})", + subtaskBaseDirectory, jobID, jobVertexID, subtaskIndex, e); } } }, @@ -240,27 +260,32 @@ private void syncDiscardLocalStateForCollection(CollectionReported state is tagged by clients so that this class can properly forward to the right receiver for the * checkpointed state. - * - * TODO: all interaction with local state store must still be implemented! It is currently just a placeholder. */ public class TaskStateManagerImpl implements TaskStateManager { + /** The logger for this class. */ + private static final Logger LOG = LoggerFactory.getLogger(TaskStateManagerImpl.class); + /** The id of the job for which this manager was created, can report, and recover. */ private final JobID jobId; @@ -117,21 +122,27 @@ public PrioritizedOperatorSubtaskState prioritizedOperatorState(OperatorID opera TaskStateSnapshot localStateSnapshot = localStateStore.retrieveLocalState(jobManagerTaskRestore.getRestoreCheckpointId()); + List alternativesByPriority = Collections.emptyList(); + if (localStateSnapshot != null) { OperatorSubtaskState localSubtaskState = localStateSnapshot.getSubtaskStateByOperatorID(operatorID); if (localSubtaskState != null) { - PrioritizedOperatorSubtaskState.Builder builder = new PrioritizedOperatorSubtaskState.Builder( - jobManagerSubtaskState, - Collections.singletonList(localSubtaskState)); - return builder.build(); + alternativesByPriority = Collections.singletonList(localSubtaskState); } } + if (LOG.isTraceEnabled()) { + LOG.trace("Operator {} has remote state {} from job manager and local state alternatives {} from local " + + "state store {}.", + operatorID, jobManagerSubtaskState, alternativesByPriority, localStateStore); + } + PrioritizedOperatorSubtaskState.Builder builder = new PrioritizedOperatorSubtaskState.Builder( jobManagerSubtaskState, - Collections.emptyList(), + alternativesByPriority, true); + return builder.build(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java index 09c9efb695994..82082e0f8f032 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PrioritizedOperatorSubtaskStateTest.java @@ -216,7 +216,7 @@ private OperatorSubtaskState createAlternativeSubtaskState(OperatorSubtaskState private boolean checkResultAsExpected( Function> extractor, - Function>> extractor2, + Function>> extractor2, PrioritizedOperatorSubtaskState prioritizedResult, OperatorSubtaskState... expectedOrdered) { @@ -226,7 +226,7 @@ private boolean checkResultAsExpected( } return checkRepresentSameOrder( - extractor2.apply(prioritizedResult), + extractor2.apply(prioritizedResult).iterator(), collector.toArray(new StateObjectCollection[collector.size()])); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskStateManagerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskStateManagerImplTest.java index 926c1961c6652..f58f3f442ab25 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskStateManagerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskStateManagerImplTest.java @@ -142,7 +142,7 @@ public void testStateReportingAndRetrieving() { // checks for operator 1. Iterator> prioritizedManagedKeyedState_1 = - prioritized_1.getPrioritizedManagedKeyedState(); + prioritized_1.getPrioritizedManagedKeyedState().iterator(); Assert.assertTrue(prioritizedManagedKeyedState_1.hasNext()); StateObjectCollection current = prioritizedManagedKeyedState_1.next(); @@ -158,7 +158,7 @@ public void testStateReportingAndRetrieving() { // checks for operator 2. Iterator> prioritizedRawKeyedState_2 = - prioritized_2.getPrioritizedRawKeyedState(); + prioritized_2.getPrioritizedRawKeyedState().iterator(); Assert.assertTrue(prioritizedRawKeyedState_2.hasNext()); current = prioritizedRawKeyedState_2.next(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java index ba27a0a27dd4d..dd75fb298ca0c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedure.java @@ -36,7 +36,7 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; -import java.util.Iterator; +import java.util.List; /** * This class implements the logic that creates (and potentially restores) a state backend. The restore logic @@ -62,6 +62,9 @@ public class BackendRestorerProcedure< /** This registry is used so that recovery can participate in the task lifecycle, i.e. can be canceled. */ private final CloseableRegistry backendCloseableRegistry; + /** Description of this instance for logging. */ + private final String logDescription; + /** * Creates a new backend restorer using the given backend supplier and the closeable registry. * @@ -70,43 +73,63 @@ public class BackendRestorerProcedure< */ public BackendRestorerProcedure( @Nonnull SupplierWithException instanceSupplier, - @Nonnull CloseableRegistry backendCloseableRegistry) { + @Nonnull CloseableRegistry backendCloseableRegistry, + @Nonnull String logDescription) { this.instanceSupplier = Preconditions.checkNotNull(instanceSupplier); this.backendCloseableRegistry = Preconditions.checkNotNull(backendCloseableRegistry); + this.logDescription = logDescription; } /** * Creates a new state backend and restores it from the provided set of state snapshot alternatives. * - * @param restoreOptions iterator over a prioritized set of state snapshot alternatives for recovery. + * @param restoreOptions list of prioritized state snapshot alternatives for recovery. * @return the created (and restored) state backend. * @throws Exception if the backend could not be created or restored. */ - public @Nonnull - T createAndRestore(@Nonnull Iterator> restoreOptions) throws Exception { + @Nonnull + public T createAndRestore(@Nonnull List> restoreOptions) throws Exception { + + if (restoreOptions.isEmpty()) { + restoreOptions = Collections.singletonList(Collections.emptyList()); + } + + int alternativeIdx = 0; + + Exception collectedException = null; + + while (alternativeIdx < restoreOptions.size()) { + + Collection restoreState = restoreOptions.get(alternativeIdx); - // This ensures that we always call the restore method even if there is no previous state - // (required by some backends). - Collection attemptState = restoreOptions.hasNext() ? - restoreOptions.next() : - Collections.emptyList(); + ++alternativeIdx; + + if (restoreState.isEmpty()) { + LOG.debug("Creating {} with empty state.", logDescription); + } else { + if (LOG.isTraceEnabled()) { + LOG.trace("Creating {} and restoring with state {} from alternative ({}/{}).", + logDescription, restoreState, alternativeIdx, restoreOptions.size()); + } else { + LOG.debug("Creating {} and restoring with state from alternative ({}/{}).", + logDescription, alternativeIdx, restoreOptions.size()); + } + } - while (true) { try { - return attemptCreateAndRestore(attemptState); + return attemptCreateAndRestore(restoreState); } catch (Exception ex) { - // more attempts? - if (restoreOptions.hasNext()) { - attemptState = restoreOptions.next(); - LOG.warn("Exception while restoring backend, will retry with another snapshot replica.", ex); - } else { + collectedException = ExceptionUtils.firstOrSuppressed(ex, collectedException); - throw new FlinkException("Could not restore from any of the provided restore options.", ex); - } + LOG.warn("Exception while restoring {} from alternative ({}/{}), will retry while more " + + "alternatives are available.", logDescription, alternativeIdx, restoreOptions.size(), ex); } } + + throw new FlinkException("Could not restore " + logDescription + " from any of the " + restoreOptions.size() + + " provided restore options.", collectedException); } private T attemptCreateAndRestore(Collection restoreState) throws Exception { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index 11e2dda82ea72..acbc2f8bc9ed5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -143,11 +143,11 @@ public StreamOperatorStateContext streamOperatorStateContext( // -------------- Raw State Streams -------------- rawKeyedStateInputs = rawKeyedStateInputs( - prioritizedOperatorSubtaskStates.getPrioritizedRawKeyedState()); + prioritizedOperatorSubtaskStates.getPrioritizedRawKeyedState().iterator()); streamTaskCloseableRegistry.registerCloseable(rawKeyedStateInputs); rawOperatorStateInputs = rawOperatorStateInputs( - prioritizedOperatorSubtaskStates.getPrioritizedRawOperatorState()); + prioritizedOperatorSubtaskStates.getPrioritizedRawOperatorState().iterator()); streamTaskCloseableRegistry.registerCloseable(rawOperatorStateInputs); // -------------- Internal Timer Service Manager -------------- @@ -226,12 +226,16 @@ protected OperatorStateBackend operatorStateBackend( PrioritizedOperatorSubtaskState prioritizedOperatorSubtaskStates, CloseableRegistry backendCloseableRegistry) throws Exception { + String logDescription = "operator state backend for " + operatorIdentifierText; + BackendRestorerProcedure backendRestorer = new BackendRestorerProcedure<>( () -> stateBackend.createOperatorStateBackend(environment, operatorIdentifierText), - backendCloseableRegistry); + backendCloseableRegistry, + logDescription); - return backendRestorer.createAndRestore(prioritizedOperatorSubtaskStates.getPrioritizedManagedOperatorState()); + return backendRestorer.createAndRestore( + prioritizedOperatorSubtaskStates.getPrioritizedManagedOperatorState()); } protected AbstractKeyedStateBackend keyedStatedBackend( @@ -244,6 +248,8 @@ protected AbstractKeyedStateBackend keyedStatedBackend( return null; } + String logDescription = "keyed state backend for " + operatorIdentifierText; + TaskInfo taskInfo = environment.getTaskInfo(); final KeyGroupRange keyGroupRange = KeyGroupRangeAssignment.computeKeyGroupRangeForOperatorIndex( @@ -261,9 +267,11 @@ protected AbstractKeyedStateBackend keyedStatedBackend( taskInfo.getMaxNumberOfParallelSubtasks(), keyGroupRange, environment.getTaskKvStateRegistry()), - backendCloseableRegistry); + backendCloseableRegistry, + logDescription); - return backendRestorer.createAndRestore(prioritizedOperatorSubtaskStates.getPrioritizedManagedKeyedState()); + return backendRestorer.createAndRestore( + prioritizedOperatorSubtaskStates.getPrioritizedManagedKeyedState()); } protected CloseableIterable rawOperatorStateInputs( diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedureTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedureTest.java index 2126f707a621e..0f15d110280a9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedureTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/BackendRestorerProcedureTest.java @@ -104,21 +104,17 @@ public void testRestoreProcedureOrderAndFailure() throws Exception { new StateObjectCollection<>(Collections.singletonList(firstFailHandle)), new StateObjectCollection<>(Collections.singletonList(secondSuccessHandle)), new StateObjectCollection<>(Collections.singletonList(thirdNotUsedHandle))); - Iterator> iterator = sortedRestoreOptions.iterator(); BackendRestorerProcedure restorerProcedure = - new BackendRestorerProcedure<>(backendSupplier, closeableRegistry); + new BackendRestorerProcedure<>(backendSupplier, closeableRegistry, "test op state backend"); - OperatorStateBackend restoredBackend = restorerProcedure.createAndRestore(iterator); + OperatorStateBackend restoredBackend = restorerProcedure.createAndRestore(sortedRestoreOptions); Assert.assertNotNull(restoredBackend); try { - Assert.assertTrue(iterator.hasNext()); - Assert.assertTrue(thirdNotUsedHandle == iterator.next().iterator().next()); verify(firstFailHandle).openInputStream(); verify(secondSuccessHandle).openInputStream(); verifyZeroInteractions(thirdNotUsedHandle); - Assert.assertFalse(iterator.hasNext()); ListState listState = restoredBackend.getListState(stateDescriptor); @@ -151,13 +147,12 @@ public void testExceptionThrownIfAllRestoresFailed() throws Exception { new StateObjectCollection<>(Collections.singletonList(firstFailHandle)), new StateObjectCollection<>(Collections.singletonList(secondFailHandle)), new StateObjectCollection<>(Collections.singletonList(thirdFailHandle))); - Iterator> iterator = sortedRestoreOptions.iterator(); BackendRestorerProcedure restorerProcedure = - new BackendRestorerProcedure<>(backendSupplier, closeableRegistry); + new BackendRestorerProcedure<>(backendSupplier, closeableRegistry, "test op state backend"); try { - restorerProcedure.createAndRestore(iterator); + restorerProcedure.createAndRestore(sortedRestoreOptions); Assert.fail(); } catch (Exception ignore) { } @@ -165,7 +160,6 @@ public void testExceptionThrownIfAllRestoresFailed() throws Exception { verify(firstFailHandle).openInputStream(); verify(secondFailHandle).openInputStream(); verify(thirdFailHandle).openInputStream(); - Assert.assertFalse(iterator.hasNext()); } /** @@ -183,12 +177,12 @@ public void testCanBeCanceledViaRegistry() throws Exception { Collections.singletonList(new StateObjectCollection<>(Collections.singletonList(blockingRestoreHandle))); BackendRestorerProcedure restorerProcedure = - new BackendRestorerProcedure<>(backendSupplier, closeableRegistry); + new BackendRestorerProcedure<>(backendSupplier, closeableRegistry, "test op state backend"); AtomicReference exceptionReference = new AtomicReference<>(null); Thread restoreThread = new Thread(() -> { try { - restorerProcedure.createAndRestore(sortedRestoreOptions.iterator()); + restorerProcedure.createAndRestore(sortedRestoreOptions); } catch (Exception e) { exceptionReference.set(e); } From e9e13dec10f1a1ee57c46719d758885c4f33dcf3 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 27 Feb 2018 16:53:03 +0100 Subject: [PATCH 002/268] [hotfix] [core] Suppress unused warning config options only used in shell scripts and doc generation. --- .../apache/flink/configuration/CoreOptions.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index ccce0aba8db2d..dc544e0b433d1 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -137,17 +137,32 @@ public static String[] getParentFirstLoaderPatterns(Configuration config) { .key("env.java.opts.taskmanager") .defaultValue(""); + /** + * This options is here only for documentation generation, it is only + * evaluated in the shell scripts. + */ + @SuppressWarnings("unused") public static final ConfigOption FLINK_LOG_DIR = ConfigOptions .key("env.log.dir") .noDefaultValue() .withDescription("Defines the directory where the Flink logs are saved. It has to be an absolute path." + " (Defaults to the log directory under Flink’s home)"); + /** + * This options is here only for documentation generation, it is only + * evaluated in the shell scripts. + */ + @SuppressWarnings("unused") public static final ConfigOption FLINK_LOG_MAX = ConfigOptions .key("env.log.max") .defaultValue(5) .withDescription("The maximum number of old log files to keep."); + /** + * This options is here only for documentation generation, it is only + * evaluated in the shell scripts. + */ + @SuppressWarnings("unused") public static final ConfigOption FLINK_SSH_OPTIONS = ConfigOptions .key("env.ssh.opts") .noDefaultValue() From a269f8519305faff153e84d729873b6f9497bd36 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 27 Feb 2018 17:04:29 +0100 Subject: [PATCH 003/268] [FLINK-8798] [core] Make force 'commons-logging' to be parent-first loaded. --- .../main/java/org/apache/flink/configuration/CoreOptions.java | 4 ++-- .../apache/flink/configuration/ParentFirstPatternsTest.java | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index dc544e0b433d1..8ac729d94f7f4 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -80,14 +80,14 @@ public class CoreOptions { * and formats (flink-avro, etc) are loaded parent-first as well if they are in the * core classpath. *
  • Java annotations and loggers, defined by the following list: - * javax.annotation;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback. + * javax.annotation;org.slf4j;org.apache.log4j;org.apache.logging;org.apache.commons.logging;ch.qos.logback. * This is done for convenience, to avoid duplication of annotations and multiple * log bindings.
  • * */ public static final ConfigOption ALWAYS_PARENT_FIRST_LOADER_PATTERNS = ConfigOptions .key("classloader.parent-first-patterns.default") - .defaultValue("java.;scala.;org.apache.flink.;com.esotericsoftware.kryo;org.apache.hadoop.;javax.annotation.;org.slf4j;org.apache.log4j;org.apache.logging.log4j;ch.qos.logback") + .defaultValue("java.;scala.;org.apache.flink.;com.esotericsoftware.kryo;org.apache.hadoop.;javax.annotation.;org.slf4j;org.apache.log4j;org.apache.logging;org.apache.commons.logging;ch.qos.logback") .withDeprecatedKeys("classloader.parent-first-patterns") .withDescription("A (semicolon-separated) list of patterns that specifies which classes should always be" + " resolved through the parent ClassLoader first. A pattern is a simple prefix that is checked against" + diff --git a/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java b/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java index ca4b511f45636..b373a6dde2a97 100644 --- a/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java +++ b/flink-core/src/test/java/org/apache/flink/configuration/ParentFirstPatternsTest.java @@ -54,7 +54,8 @@ public void testAllCorePatterns() { public void testLoggersParentFirst() { assertTrue(PARENT_FIRST_PACKAGES.contains("org.slf4j")); assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.log4j")); - assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.logging.log4j")); + assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.logging")); + assertTrue(PARENT_FIRST_PACKAGES.contains("org.apache.commons.logging")); assertTrue(PARENT_FIRST_PACKAGES.contains("ch.qos.logback")); } From 1d26062de130c05fdbe7701b55766b4a8d433418 Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Mon, 12 Feb 2018 18:11:36 +0800 Subject: [PATCH 004/268] [FLINK-8538][table]Add a Kafka table source factory with JSON format support --- .../kafka/Kafka010JsonTableSourceFactory.java | 36 +++ ...che.flink.table.sources.TableSourceFactory | 16 ++ .../resources/tableSourceConverter.properties | 29 +++ .../kafka/Kafka010TableSourceFactoryTest.java | 41 ++++ .../kafka/Kafka011JsonTableSourceFactory.java | 36 +++ ...che.flink.table.sources.TableSourceFactory | 16 ++ .../resources/tableSourceConverter.properties | 29 +++ .../kafka/Kafka011TableSourceFactoryTest.java | 41 ++++ .../kafka/Kafka08JsonTableSourceFactory.java | 36 +++ ...che.flink.table.sources.TableSourceFactory | 16 ++ .../resources/tableSourceConverter.properties | 29 +++ .../kafka/Kafka08TableSourceFactoryTest.java | 42 ++++ .../kafka/Kafka09JsonTableSourceFactory.java | 36 +++ ...che.flink.table.sources.TableSourceFactory | 16 ++ .../resources/tableSourceConverter.properties | 29 +++ .../kafka/Kafka09TableSourceFactoryTest.java | 41 ++++ .../flink-connector-kafka-base/pom.xml | 14 ++ .../kafka/KafkaJsonTableSource.java | 17 ++ .../kafka/KafkaJsonTableSourceFactory.java | 227 ++++++++++++++++++ .../connectors/kafka/KafkaTableSource.java | 30 +++ .../apache/flink/table/descriptors/Kafka.java | 199 +++++++++++++++ .../table/descriptors/KafkaValidator.java | 193 +++++++++++++++ .../KafkaJsonTableFromDescriptorTestBase.java | 127 ++++++++++ .../src/test/resources/kafka-json-schema.json | 35 +++ .../apache/flink/table/api/TableSchema.scala | 5 + 25 files changed, 1336 insertions(+) create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/main/resources/tableSourceConverter.properties create mode 100644 flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/resources/tableSourceConverter.properties create mode 100644 flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java create mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory create mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/resources/tableSourceConverter.properties create mode 100644 flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/resources/tableSourceConverter.properties create mode 100644 flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableFromDescriptorTestBase.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-json-schema.json diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java new file mode 100644 index 0000000000000..1d03f6c4e8c03 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java @@ -0,0 +1,36 @@ +/* + * 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.streaming.connectors.kafka; + +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_010; + +/** + * Factory for creating configured instances of {@link Kafka010JsonTableSource}. + */ +public class Kafka010JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + @Override + protected KafkaJsonTableSource.Builder createBuilder() { + return new Kafka010JsonTableSource.Builder(); + } + + @Override + protected String kafkaVersion() { + return KAFKA_VERSION_VALUE_010; + } +} diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory new file mode 100644 index 0000000000000..9ef54fcb045fa --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.streaming.connectors.kafka.Kafka010JsonTableSourceFactory diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/tableSourceConverter.properties b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/tableSourceConverter.properties new file mode 100644 index 0000000000000..5409b49703088 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/tableSourceConverter.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +################################################################################ +# The config file is used to specify the packages of current module where +# to find TableSourceConverter implementation class annotated with TableType. +# If there are multiple packages to scan, put those packages together into a +# string separated with ',', for example, org.package1,org.package2. +# Please notice: +# It's better to have a tableSourceConverter.properties in each connector Module +# which offers converters instead of put all information into the +# tableSourceConverter.properties of flink-table module. +################################################################################ +scan.packages=org.apache.flink.streaming.connectors.kafka diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java new file mode 100644 index 0000000000000..15b89e8e2f85a --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java @@ -0,0 +1,41 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.table.descriptors.Kafka; + +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_010; + +/** + * Tests for {@link Kafka010JsonTableSourceFactory}. + */ +public class Kafka010TableSourceFactoryTest extends KafkaJsonTableFromDescriptorTestBase { + protected String versionForTest() { + return KAFKA_VERSION_VALUE_010; + } + + protected KafkaJsonTableSource.Builder builderForTest() { + return Kafka010JsonTableSource.builder(); + } + + @Override + protected void extraSettings(KafkaTableSource.Builder builder, Kafka kafka) { + // no extra settings + } +} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java new file mode 100644 index 0000000000000..ca4d6ce01374d --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java @@ -0,0 +1,36 @@ +/* + * 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.streaming.connectors.kafka; + +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_011; + +/** + * Factory for creating configured instances of {@link Kafka011JsonTableSource}. + */ +public class Kafka011JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + @Override + protected KafkaJsonTableSource.Builder createBuilder() { + return new Kafka011JsonTableSource.Builder(); + } + + @Override + protected String kafkaVersion() { + return KAFKA_VERSION_VALUE_011; + } +} diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory new file mode 100644 index 0000000000000..75135e57d746c --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.streaming.connectors.kafka.Kafka011JsonTableSourceFactory diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/tableSourceConverter.properties b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/tableSourceConverter.properties new file mode 100644 index 0000000000000..5409b49703088 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/tableSourceConverter.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +################################################################################ +# The config file is used to specify the packages of current module where +# to find TableSourceConverter implementation class annotated with TableType. +# If there are multiple packages to scan, put those packages together into a +# string separated with ',', for example, org.package1,org.package2. +# Please notice: +# It's better to have a tableSourceConverter.properties in each connector Module +# which offers converters instead of put all information into the +# tableSourceConverter.properties of flink-table module. +################################################################################ +scan.packages=org.apache.flink.streaming.connectors.kafka diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java new file mode 100644 index 0000000000000..84ac39b6f47e8 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java @@ -0,0 +1,41 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.table.descriptors.Kafka; + +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_011; + +/** + * Tests for {@link Kafka011JsonTableSourceFactory}. + */ +public class Kafka011TableSourceFactoryTest extends KafkaJsonTableFromDescriptorTestBase { + protected String versionForTest() { + return KAFKA_VERSION_VALUE_011; + } + + protected KafkaJsonTableSource.Builder builderForTest() { + return Kafka011JsonTableSource.builder(); + } + + @Override + protected void extraSettings(KafkaTableSource.Builder builder, Kafka kafka) { + // no extra settings + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java new file mode 100644 index 0000000000000..e4e50960b627c --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java @@ -0,0 +1,36 @@ +/* + * 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.streaming.connectors.kafka; + +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_08; + +/** + * Factory for creating configured instances of {@link Kafka08JsonTableSource}. + */ +public class Kafka08JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + @Override + protected KafkaJsonTableSource.Builder createBuilder() { + return new Kafka08JsonTableSource.Builder(); + } + + @Override + protected String kafkaVersion() { + return KAFKA_VERSION_VALUE_08; + } +} diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory new file mode 100644 index 0000000000000..9092955842e7a --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.streaming.connectors.kafka.Kafka08JsonTableSourceFactory diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/tableSourceConverter.properties b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/tableSourceConverter.properties new file mode 100644 index 0000000000000..5409b49703088 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/tableSourceConverter.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +################################################################################ +# The config file is used to specify the packages of current module where +# to find TableSourceConverter implementation class annotated with TableType. +# If there are multiple packages to scan, put those packages together into a +# string separated with ',', for example, org.package1,org.package2. +# Please notice: +# It's better to have a tableSourceConverter.properties in each connector Module +# which offers converters instead of put all information into the +# tableSourceConverter.properties of flink-table module. +################################################################################ +scan.packages=org.apache.flink.streaming.connectors.kafka diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java new file mode 100644 index 0000000000000..a2edc09ec125f --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java @@ -0,0 +1,42 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.table.descriptors.Kafka; + +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_08; + +/** + * Tests for {@link Kafka08JsonTableSourceFactory}. + */ +public class Kafka08TableSourceFactoryTest extends KafkaJsonTableFromDescriptorTestBase { + protected String versionForTest() { + return KAFKA_VERSION_VALUE_08; + } + + protected KafkaJsonTableSource.Builder builderForTest() { + return Kafka08JsonTableSource.builder(); + } + + @Override + protected void extraSettings(KafkaTableSource.Builder builder, Kafka kafka) { + builder.getKafkaProps().put("zookeeper.connect", "localhost:1111"); + kafka.zookeeperConnect("localhost:1111"); + } +} diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java new file mode 100644 index 0000000000000..bbda4ae66d636 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java @@ -0,0 +1,36 @@ +/* + * 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.streaming.connectors.kafka; + +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_09; + +/** + * Factory for creating configured instances of {@link Kafka09JsonTableSource}. + */ +public class Kafka09JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + @Override + protected KafkaJsonTableSource.Builder createBuilder() { + return new Kafka09JsonTableSource.Builder(); + } + + @Override + protected String kafkaVersion() { + return KAFKA_VERSION_VALUE_09; + } +} diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory new file mode 100644 index 0000000000000..2f38bd0e9746d --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/META-INF/services/org.apache.flink.table.sources.TableSourceFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.streaming.connectors.kafka.Kafka09JsonTableSourceFactory diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/tableSourceConverter.properties b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/tableSourceConverter.properties new file mode 100644 index 0000000000000..5409b49703088 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/tableSourceConverter.properties @@ -0,0 +1,29 @@ +# +# 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. +# + +################################################################################ +# The config file is used to specify the packages of current module where +# to find TableSourceConverter implementation class annotated with TableType. +# If there are multiple packages to scan, put those packages together into a +# string separated with ',', for example, org.package1,org.package2. +# Please notice: +# It's better to have a tableSourceConverter.properties in each connector Module +# which offers converters instead of put all information into the +# tableSourceConverter.properties of flink-table module. +################################################################################ +scan.packages=org.apache.flink.streaming.connectors.kafka diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java new file mode 100644 index 0000000000000..fc85ea77a07cd --- /dev/null +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java @@ -0,0 +1,41 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.table.descriptors.Kafka; + +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_09; + +/** + * Factory for creating configured instances of {@link Kafka09JsonTableSource}. + */ +public class Kafka09TableSourceFactoryTest extends KafkaJsonTableFromDescriptorTestBase { + protected String versionForTest() { + return KAFKA_VERSION_VALUE_09; + } + + protected KafkaJsonTableSource.Builder builderForTest() { + return Kafka09JsonTableSource.builder(); + } + + @Override + protected void extraSettings(KafkaTableSource.Builder builder, Kafka kafka) { + // no extra settings + } +} diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml index e7412cf1d5222..2ccaa2e5c1d3e 100644 --- a/flink-connectors/flink-connector-kafka-base/pom.xml +++ b/flink-connectors/flink-connector-kafka-base/pom.xml @@ -205,6 +205,20 @@ under the License. test + + org.apache.flink + flink-scala_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-streaming-scala_${scala.binary.version} + ${project.version} + test + + diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java index f581e89830bf6..d2dafe72b3495 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java @@ -27,6 +27,7 @@ import org.apache.flink.table.sources.StreamTableSource; import java.util.Map; +import java.util.Objects; import java.util.Properties; /** @@ -86,6 +87,22 @@ public String explainSource() { return "KafkaJSONTableSource"; } + @Override + public boolean equals(Object other) { + if (super.equals(other)) { + KafkaJsonTableSource otherSource = (KafkaJsonTableSource) other; + return Objects.equals(failOnMissingField, otherSource.failOnMissingField) + && Objects.equals(jsonSchema, otherSource.jsonSchema) + && Objects.equals(fieldMapping, otherSource.fieldMapping); + } + return false; + } + + @Override + public int hashCode() { + return 31 * super.hashCode() + Objects.hash(failOnMissingField, jsonSchema, fieldMapping); + } + //////// SETTERS FOR OPTIONAL PARAMETERS /** diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java new file mode 100644 index 0000000000000..918b83357ff56 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java @@ -0,0 +1,227 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.json.JsonSchemaConverter; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.JsonValidator; +import org.apache.flink.table.descriptors.KafkaValidator; +import org.apache.flink.table.descriptors.SchemaValidator; +import org.apache.flink.table.sources.TableSource; +import org.apache.flink.table.sources.TableSourceFactory; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_VERSION; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA_STRING; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_TYPE_VALUE; +import static org.apache.flink.table.descriptors.KafkaValidator.BOOTSTRAP_SERVERS; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TYPE_VALUE; +import static org.apache.flink.table.descriptors.KafkaValidator.GROUP_ID; +import static org.apache.flink.table.descriptors.KafkaValidator.JSON_FIELD; +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION; +import static org.apache.flink.table.descriptors.KafkaValidator.OFFSET; +import static org.apache.flink.table.descriptors.KafkaValidator.PARTITION; +import static org.apache.flink.table.descriptors.KafkaValidator.SPECIFIC_OFFSETS; +import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE; +import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE_VALUE_EARLIEST; +import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE_VALUE_GROUP_OFFSETS; +import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE_VALUE_LATEST; +import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE_VALUE_SPECIFIC_OFFSETS; +import static org.apache.flink.table.descriptors.KafkaValidator.TABLE_FIELD; +import static org.apache.flink.table.descriptors.KafkaValidator.TABLE_JSON_MAPPING; +import static org.apache.flink.table.descriptors.KafkaValidator.TOPIC; +import static org.apache.flink.table.descriptors.KafkaValidator.ZOOKEEPER_CONNECT; +import static org.apache.flink.table.descriptors.SchemaValidator.PROCTIME; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_VERSION; + +import scala.Option; +import scala.collection.JavaConversions; + +/** + * Factory for creating configured instances of {@link KafkaJsonTableSource}. + */ +public abstract class KafkaJsonTableSourceFactory implements TableSourceFactory { + @Override + public Map requiredContext() { + Map context = new HashMap<>(); + context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE); // kafka connector + context.put(FORMAT_TYPE(), FORMAT_TYPE_VALUE()); // Json format + context.put(KAFKA_VERSION, kafkaVersion()); // for different implementations + context.put(CONNECTOR_VERSION(), "1"); + context.put(FORMAT_VERSION(), "1"); + context.put(SCHEMA_VERSION(), "1"); + return context; + } + + @Override + public List supportedProperties() { + List properties = new ArrayList<>(); + + // kafka + properties.add(KAFKA_VERSION); + properties.add(BOOTSTRAP_SERVERS); + properties.add(GROUP_ID); + properties.add(ZOOKEEPER_CONNECT); + properties.add(TOPIC); + properties.add(STARTUP_MODE); + properties.add(SPECIFIC_OFFSETS + ".#." + PARTITION); + properties.add(SPECIFIC_OFFSETS + ".#." + OFFSET); + + // json format + properties.add(FORMAT_SCHEMA_STRING()); + properties.add(FORMAT_FAIL_ON_MISSING_FIELD()); + + // table json mapping + properties.add(TABLE_JSON_MAPPING + ".#." + TABLE_FIELD); + properties.add(TABLE_JSON_MAPPING + ".#." + JSON_FIELD); + + // schema + properties.add(SCHEMA() + ".#." + DescriptorProperties.TYPE()); + properties.add(SCHEMA() + ".#." + DescriptorProperties.NAME()); + + // time attributes + properties.add(SCHEMA() + ".#." + PROCTIME()); +// properties.add(SCHEMA() + ".#." + ROWTIME() + ".#." + TIMESTAMPS_CLASS()); +// properties.add(SCHEMA() + ".#." + ROWTIME() + ".#." + TIMESTAMPS_TYPE()); + + return properties; + } + + @Override + public TableSource create(Map properties) { + DescriptorProperties params = new DescriptorProperties(true); + params.putProperties(properties); + + // validate + new KafkaValidator().validate(params); + new JsonValidator().validate(params); + new SchemaValidator(true).validate(params); + + // build + KafkaJsonTableSource.Builder builder = createBuilder(); + Properties kafkaProps = new Properties(); + + // Set the required parameters. + String topic = params.getString(TOPIC).get(); + TableSchema tableSchema = params.getTableSchema(SCHEMA()).get(); + + kafkaProps.put(BOOTSTRAP_SERVERS, params.getString(BOOTSTRAP_SERVERS).get()); + kafkaProps.put(GROUP_ID, params.getString(GROUP_ID).get()); + + // Set the zookeeper connect for kafka 0.8. + Option zkConnect = params.getString(ZOOKEEPER_CONNECT); + if (zkConnect.isDefined()) { + kafkaProps.put(ZOOKEEPER_CONNECT, zkConnect.get()); + } + + builder.withKafkaProperties(kafkaProps).forTopic(topic).withSchema(tableSchema); + + // Set the startup mode. + String startupMode = params.getString(STARTUP_MODE).get(); + if (null != startupMode) { + switch (startupMode) { + case STARTUP_MODE_VALUE_EARLIEST: + builder.fromEarliest(); + break; + case STARTUP_MODE_VALUE_LATEST: + builder.fromLatest(); + break; + case STARTUP_MODE_VALUE_GROUP_OFFSETS: + builder.fromGroupOffsets(); + break; + case STARTUP_MODE_VALUE_SPECIFIC_OFFSETS: + Map partitions = JavaConversions. + mapAsJavaMap(params.getIndexedProperty(SPECIFIC_OFFSETS, PARTITION)); + Map offsetMap = new HashMap<>(); + for (int i = 0; i < partitions.size(); i++) { + offsetMap.put( + new KafkaTopicPartition( + topic, + Integer.valueOf(params.getString( + SPECIFIC_OFFSETS + "" + "." + i + "." + PARTITION).get())), + Long.valueOf(params.getString( + SPECIFIC_OFFSETS + "" + "." + i + "." + OFFSET).get())); + } + builder.fromSpecificOffsets(offsetMap); + break; + } + } + + // Set whether fail on missing JSON field. + Option failOnMissing = params.getString(FORMAT_FAIL_ON_MISSING_FIELD()); + if (failOnMissing.isDefined()) { + builder.failOnMissingField(Boolean.valueOf(failOnMissing.get())); + } + + // Set the JSON schema. + Option jsonSchema = params.getString(FORMAT_SCHEMA_STRING()); + if (jsonSchema.isDefined()) { + TypeInformation jsonSchemaType = JsonSchemaConverter.convert(jsonSchema.get()); + builder.forJsonSchema(TableSchema.fromTypeInfo(jsonSchemaType)); + } + + // Set the table => JSON fields mapping. + Map mappingTableFields = JavaConversions. + mapAsJavaMap(params.getIndexedProperty(TABLE_JSON_MAPPING, TABLE_FIELD)); + + if (!mappingTableFields.isEmpty()) { + Map tableJsonMapping = new HashMap<>(); + for (int i = 0; i < mappingTableFields.size(); i++) { + tableJsonMapping.put(params.getString(TABLE_JSON_MAPPING + "." + i + "." + TABLE_FIELD).get(), + params.getString(TABLE_JSON_MAPPING + "." + i + "." + JSON_FIELD).get() + ); + } + builder.withTableToJsonMapping(tableJsonMapping); + } + + // Set the time attributes. + setTimeAttributes(tableSchema, params, builder); + + return builder.build(); + } + + protected abstract KafkaJsonTableSource.Builder createBuilder(); + + protected abstract String kafkaVersion(); + + private void setTimeAttributes(TableSchema schema, DescriptorProperties params, KafkaJsonTableSource.Builder builder) { + // TODO to deal with rowtime fields + Option proctimeField; + for (int i = 0; i < schema.getColumnNum(); i++) { + proctimeField = params.getString(SCHEMA() + "." + i + "." + PROCTIME()); + if (proctimeField.isDefined()) { + builder.withProctimeAttribute(schema.getColumnName(i).get()); + } + } + } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index d5cda4ab8daa7..9ce3b8ed5a17c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -42,6 +42,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import scala.Option; @@ -138,6 +139,35 @@ public String explainSource() { return TableConnectorUtil.generateRuntimeName(this.getClass(), schema.getColumnNames()); } + @Override + public boolean equals(Object o) { + if (!o.getClass().equals(this.getClass())) { + return false; + } + KafkaTableSource other = (KafkaTableSource) o; + return Objects.equals(topic, other.topic) + && Objects.equals(schema, other.schema) + && Objects.equals(properties, other.properties) + && Objects.equals(proctimeAttribute, other.proctimeAttribute) + && Objects.equals(returnType, other.returnType) + && Objects.equals(rowtimeAttributeDescriptors, other.rowtimeAttributeDescriptors) + && Objects.equals(specificStartupOffsets, other.specificStartupOffsets) + && Objects.equals(startupMode, other.startupMode); + } + + @Override + public int hashCode() { + return Objects.hash( + topic, + schema, + properties, + proctimeAttribute, + returnType, + rowtimeAttributeDescriptors, + specificStartupOffsets, + startupMode); + } + /** * Returns a version-specific Kafka consumer with the start position configured. * diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java new file mode 100644 index 0000000000000..4733f6e002db8 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java @@ -0,0 +1,199 @@ +/* + * 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.table.descriptors; + +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; + +import static org.apache.flink.table.descriptors.KafkaValidator.BOOTSTRAP_SERVERS; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TYPE_VALUE; +import static org.apache.flink.table.descriptors.KafkaValidator.GROUP_ID; +import static org.apache.flink.table.descriptors.KafkaValidator.JSON_FIELD; +import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION; +import static org.apache.flink.table.descriptors.KafkaValidator.OFFSET; +import static org.apache.flink.table.descriptors.KafkaValidator.PARTITION; +import static org.apache.flink.table.descriptors.KafkaValidator.SPECIFIC_OFFSETS; +import static org.apache.flink.table.descriptors.KafkaValidator.TABLE_FIELD; +import static org.apache.flink.table.descriptors.KafkaValidator.TABLE_JSON_MAPPING; +import static org.apache.flink.table.descriptors.KafkaValidator.TOPIC; +import static org.apache.flink.table.descriptors.KafkaValidator.ZOOKEEPER_CONNECT; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import scala.collection.JavaConversions; +import scala.collection.Seq; + +/** + * Connector descriptor for the kafka message queue. + */ +public class Kafka extends ConnectorDescriptor { + + private Optional version = Optional.empty(); + private Optional bootstrapServers = Optional.empty(); + private Optional groupId = Optional.empty(); + private Optional topic = Optional.empty(); + private Optional zookeeperConnect = Optional.empty(); + private Optional> tableJsonMapping = Optional.empty(); + + private Optional startupMode = Optional.empty(); + private Optional> specificOffsets = Optional.empty(); + + public Kafka() { + super(CONNECTOR_TYPE_VALUE, 1); + } + + /** + * Sets the kafka version. + * + * @param version + * Could be {@link KafkaValidator#KAFKA_VERSION_VALUE_011}, + * {@link KafkaValidator#KAFKA_VERSION_VALUE_010}, + * {@link KafkaValidator#KAFKA_VERSION_VALUE_09}, + * or {@link KafkaValidator#KAFKA_VERSION_VALUE_08}. + */ + public Kafka version(String version) { + this.version = Optional.of(version); + return this; + } + + /** + * Sets the bootstrap servers for kafka. + */ + public Kafka bootstrapServers(String bootstrapServers) { + this.bootstrapServers = Optional.of(bootstrapServers); + return this; + } + + /** + * Sets the consumer group id. + */ + public Kafka groupId(String groupId) { + this.groupId = Optional.of(groupId); + return this; + } + + /** + * Sets the topic to consume. + */ + public Kafka topic(String topic) { + this.topic = Optional.of(topic); + return this; + } + + /** + * Sets the startup mode. + */ + public Kafka startupMode(StartupMode startupMode) { + this.startupMode = Optional.of(startupMode); + return this; + } + + /** + * Sets the zookeeper hosts. Only required by kafka 0.8. + */ + public Kafka zookeeperConnect(String zookeeperConnect) { + this.zookeeperConnect = Optional.of(zookeeperConnect); + return this; + } + + /** + * Sets the consume offsets for the topic set with {@link Kafka#topic(String)}. + * Only works in {@link StartupMode#SPECIFIC_OFFSETS} mode. + */ + public Kafka specificOffsets(Map specificOffsets) { + this.specificOffsets = Optional.of(specificOffsets); + return this; + } + + /** + * Sets the mapping from logical table schema to json schema. + */ + public Kafka tableJsonMapping(Map jsonTableMapping) { + this.tableJsonMapping = Optional.of(jsonTableMapping); + return this; + } + + @Override + public void addConnectorProperties(DescriptorProperties properties) { + if (version.isPresent()) { + properties.putString(KAFKA_VERSION, version.get()); + } + if (bootstrapServers.isPresent()) { + properties.putString(BOOTSTRAP_SERVERS, bootstrapServers.get()); + } + if (groupId.isPresent()) { + properties.putString(GROUP_ID, groupId.get()); + } + if (topic.isPresent()) { + properties.putString(TOPIC, topic.get()); + } + if (zookeeperConnect.isPresent()) { + properties.putString(ZOOKEEPER_CONNECT, zookeeperConnect.get()); + } + if (startupMode.isPresent()) { + Map map = KafkaValidator.normalizeStartupMode(startupMode.get()); + for (Map.Entry entry : map.entrySet()) { + properties.putString(entry.getKey(), entry.getValue()); + } + } + if (specificOffsets.isPresent()) { + List propertyKeys = new ArrayList<>(); + propertyKeys.add(PARTITION); + propertyKeys.add(OFFSET); + + List> propertyValues = new ArrayList<>(specificOffsets.get().size()); + for (Map.Entry entry : specificOffsets.get().entrySet()) { + List partitionOffset = new ArrayList<>(2); + partitionOffset.add(entry.getKey().toString()); + partitionOffset.add(entry.getValue().toString()); + propertyValues.add(JavaConversions.asScalaBuffer(partitionOffset).toSeq()); + } + properties.putIndexedFixedProperties( + SPECIFIC_OFFSETS, + JavaConversions.asScalaBuffer(propertyKeys).toSeq(), + JavaConversions.asScalaBuffer(propertyValues).toSeq() + ); + } + if (tableJsonMapping.isPresent()) { + List propertyKeys = new ArrayList<>(); + propertyKeys.add(TABLE_FIELD); + propertyKeys.add(JSON_FIELD); + + List> mappingFields = new ArrayList<>(tableJsonMapping.get().size()); + for (Map.Entry entry : tableJsonMapping.get().entrySet()) { + List singleMapping = new ArrayList<>(2); + singleMapping.add(entry.getKey()); + singleMapping.add(entry.getValue()); + mappingFields.add(JavaConversions.asScalaBuffer(singleMapping).toSeq()); + } + properties.putIndexedFixedProperties( + TABLE_JSON_MAPPING, + JavaConversions.asScalaBuffer(propertyKeys).toSeq(), + JavaConversions.asScalaBuffer(mappingFields).toSeq() + ); + } + } + + @Override + public boolean needsFormat() { + return true; + } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java new file mode 100644 index 0000000000000..a3ca22f90b912 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java @@ -0,0 +1,193 @@ +/* + * 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.table.descriptors; + +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.table.api.ValidationException; + +import java.util.HashMap; +import java.util.Map; + +import scala.Function0; +import scala.Tuple2; +import scala.collection.JavaConversions; +import scala.runtime.AbstractFunction0; +import scala.runtime.BoxedUnit; + + +/** + * The validator for {@link Kafka}. + */ +public class KafkaValidator extends ConnectorDescriptorValidator { + // fields + public static final String CONNECTOR_TYPE_VALUE = "kafka"; + public static final String KAFKA_VERSION = "kafka.version"; + public static final String BOOTSTRAP_SERVERS = "bootstrap.servers"; + public static final String GROUP_ID = "group.id"; + public static final String TOPIC = "topic"; + public static final String STARTUP_MODE = "startup.mode"; + public static final String SPECIFIC_OFFSETS = "specific.offsets"; + public static final String TABLE_JSON_MAPPING = "table.json.mapping"; + + public static final String PARTITION = "partition"; + public static final String OFFSET = "offset"; + + public static final String TABLE_FIELD = "table.field"; + public static final String JSON_FIELD = "json.field"; + + public static final String ZOOKEEPER_CONNECT = "zookeeper.connect"; // only required for 0.8 + + // values + public static final String KAFKA_VERSION_VALUE_08 = "0.8"; + public static final String KAFKA_VERSION_VALUE_09 = "0.9"; + public static final String KAFKA_VERSION_VALUE_010 = "0.10"; + public static final String KAFKA_VERSION_VALUE_011 = "0.11"; + + public static final String STARTUP_MODE_VALUE_EARLIEST = "earliest-offset"; + public static final String STARTUP_MODE_VALUE_LATEST = "latest-offset"; + public static final String STARTUP_MODE_VALUE_GROUP_OFFSETS = "group-offsets"; + public static final String STARTUP_MODE_VALUE_SPECIFIC_OFFSETS = "specific-offsets"; + + // utils + public static Map normalizeStartupMode(StartupMode startupMode) { + Map mapPair = new HashMap<>(); + switch (startupMode) { + case EARLIEST: + mapPair.put(STARTUP_MODE, STARTUP_MODE_VALUE_EARLIEST); + break; + case LATEST: + mapPair.put(STARTUP_MODE, STARTUP_MODE_VALUE_LATEST); + break; + case GROUP_OFFSETS: + mapPair.put(STARTUP_MODE, STARTUP_MODE_VALUE_GROUP_OFFSETS); + break; + case SPECIFIC_OFFSETS: + mapPair.put(STARTUP_MODE, STARTUP_MODE_VALUE_SPECIFIC_OFFSETS); + break; + } + return mapPair; + } + + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + + AbstractFunction0 emptyValidator = new AbstractFunction0() { + @Override + public BoxedUnit apply() { + return BoxedUnit.UNIT; + } + }; + + properties.validateValue(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE, false); + + AbstractFunction0 version08Validator = new AbstractFunction0() { + @Override + public BoxedUnit apply() { + properties.validateString(ZOOKEEPER_CONNECT, false, 0, Integer.MAX_VALUE); + return BoxedUnit.UNIT; + } + }; + + Map> versionValidatorMap = new HashMap<>(); + versionValidatorMap.put(KAFKA_VERSION_VALUE_08, version08Validator); + versionValidatorMap.put(KAFKA_VERSION_VALUE_09, emptyValidator); + versionValidatorMap.put(KAFKA_VERSION_VALUE_010, emptyValidator); + versionValidatorMap.put(KAFKA_VERSION_VALUE_011, emptyValidator); + properties.validateEnum( + KAFKA_VERSION, + false, + toScalaImmutableMap(versionValidatorMap) + ); + + properties.validateString(BOOTSTRAP_SERVERS, false, 1, Integer.MAX_VALUE); + properties.validateString(GROUP_ID, false, 1, Integer.MAX_VALUE); + properties.validateString(TOPIC, false, 1, Integer.MAX_VALUE); + + AbstractFunction0 specificOffsetsValidator = new AbstractFunction0() { + @Override + public BoxedUnit apply() { + Map partitions = JavaConversions.mapAsJavaMap( + properties.getIndexedProperty(SPECIFIC_OFFSETS, PARTITION)); + + Map offsets = JavaConversions.mapAsJavaMap( + properties.getIndexedProperty(SPECIFIC_OFFSETS, OFFSET)); + if (partitions.isEmpty() || offsets.isEmpty()) { + throw new ValidationException("Offsets must be set for SPECIFIC_OFFSETS mode."); + } + for (int i = 0; i < partitions.size(); ++i) { + properties.validateInt( + SPECIFIC_OFFSETS + "." + i + "." + PARTITION, + false, + 0, + Integer.MAX_VALUE); + properties.validateLong( + SPECIFIC_OFFSETS + "." + i + "." + OFFSET, + false, + 0, + Long.MAX_VALUE); + } + return BoxedUnit.UNIT; + } + }; + Map> startupModeValidatorMap = new HashMap<>(); + startupModeValidatorMap.put(STARTUP_MODE_VALUE_GROUP_OFFSETS, emptyValidator); + startupModeValidatorMap.put(STARTUP_MODE_VALUE_EARLIEST, emptyValidator); + startupModeValidatorMap.put(STARTUP_MODE_VALUE_LATEST, emptyValidator); + startupModeValidatorMap.put(STARTUP_MODE_VALUE_SPECIFIC_OFFSETS, specificOffsetsValidator); + + properties.validateEnum(STARTUP_MODE, true, toScalaImmutableMap(startupModeValidatorMap)); + validateTableJsonMapping(properties); + } + + private void validateTableJsonMapping(DescriptorProperties properties) { + Map mappingTableField = JavaConversions.mapAsJavaMap( + properties.getIndexedProperty(TABLE_JSON_MAPPING, TABLE_FIELD)); + Map mappingJsonField = JavaConversions.mapAsJavaMap( + properties.getIndexedProperty(TABLE_JSON_MAPPING, JSON_FIELD)); + + if (mappingJsonField.size() != mappingJsonField.size()) { + throw new ValidationException("Table JSON mapping must be one to one."); + } + + for (int i = 0; i < mappingTableField.size(); i++) { + properties.validateString( + TABLE_JSON_MAPPING + "." + i + "." + TABLE_FIELD, + false, + 1, + Integer.MAX_VALUE); + properties.validateString( + TABLE_JSON_MAPPING + "." + i + "." + JSON_FIELD, + false, + 1, + Integer.MAX_VALUE); + } + } + + @SuppressWarnings("unchecked") + private scala.collection.immutable.Map toScalaImmutableMap(Map javaMap) { + final java.util.List> list = new java.util.ArrayList<>(javaMap.size()); + for (final java.util.Map.Entry entry : javaMap.entrySet()) { + list.add(scala.Tuple2.apply(entry.getKey(), entry.getValue())); + } + final scala.collection.Seq> seq = + scala.collection.JavaConverters.asScalaBufferConverter(list).asScala().toSeq(); + return (scala.collection.immutable.Map) scala.collection.immutable.Map$.MODULE$.apply(seq); + } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableFromDescriptorTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableFromDescriptorTestBase.java new file mode 100644 index 0000000000000..964a62425481e --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableFromDescriptorTestBase.java @@ -0,0 +1,127 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.descriptors.Kafka; + +import org.mockito.Mockito; + +/** + * Tests for {@link KafkaJsonTableSourceFactory}. + */ +public abstract class KafkaJsonTableFromDescriptorTestBase { + private static final String GROUP_ID = "test-group"; + private static final String BOOTSTRAP_SERVERS = "localhost:1234"; + private static final String TOPIC = "test-topic"; + + protected abstract String versionForTest(); + + protected abstract KafkaJsonTableSource.Builder builderForTest(); + + protected abstract void extraSettings(KafkaTableSource.Builder builder, Kafka kafka); + + private static StreamExecutionEnvironment env = Mockito.mock(StreamExecutionEnvironment.class); + private static StreamTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); + +// @Test +// public void buildJsonTableSourceTest() throws Exception { +// final URL url = getClass().getClassLoader().getResource("kafka-json-schema.json"); +// Objects.requireNonNull(url); +// final String schema = FileUtils.readFileUtf8(new File(url.getFile())); +// +// Map tableJsonMapping = new HashMap<>(); +// tableJsonMapping.put("fruit-name", "name"); +// tableJsonMapping.put("fruit-count", "count"); +// tableJsonMapping.put("event-time", "time"); +// +// // Construct with the builder. +// Properties props = new Properties(); +// props.put("group.id", GROUP_ID); +// props.put("bootstrap.servers", BOOTSTRAP_SERVERS); +// +// Map specificOffsets = new HashMap<>(); +// specificOffsets.put(new KafkaTopicPartition(TOPIC, 0), 100L); +// specificOffsets.put(new KafkaTopicPartition(TOPIC, 1), 123L); +// +// KafkaTableSource.Builder builder = builderForTest() +// .forJsonSchema(TableSchema.fromTypeInfo(JsonSchemaConverter.convert(schema))) +// .failOnMissingField(true) +// .withTableToJsonMapping(tableJsonMapping) +// .withKafkaProperties(props) +// .forTopic(TOPIC) +// .fromSpecificOffsets(specificOffsets) +// .withSchema( +// TableSchema.builder() +// .field("fruit-name", Types.STRING) +// .field("fruit-count", Types.INT) +// .field("event-time", Types.LONG) +// .field("proc-time", Types.SQL_TIMESTAMP) +// .build()) +// .withProctimeAttribute("proc-time"); +// +// // Construct with the descriptor. +// Map offsets = new HashMap<>(); +// offsets.put(0, 100L); +// offsets.put(1, 123L); +// Kafka kafka = new Kafka() +// .version(versionForTest()) +// .groupId(GROUP_ID) +// .bootstrapServers(BOOTSTRAP_SERVERS) +// .topic(TOPIC) +// .startupMode(StartupMode.SPECIFIC_OFFSETS) +// .specificOffsets(offsets) +// .tableJsonMapping(tableJsonMapping); +// extraSettings(builder, kafka); +// +// TableSource source = tEnv +// .from(kafka) +// .withFormat( +// new Json() +// .schema(schema) +// .failOnMissingField(true)) +// .withSchema(new Schema() +// .field("fruit-name", Types.STRING) +// .field("fruit-count", Types.INT) +// .field("event-time", Types.LONG) +// .field("proc-time", Types.SQL_TIMESTAMP).proctime()) +// .toTableSource(); +// +// Assert.assertEquals(builder.build(), source); +// } + +// @Test(expected = TableException.class) +// public void buildJsonTableSourceFailTest() { +// tEnv.from( +// new Kafka() +// .version(versionForTest()) +// .groupId(GROUP_ID) +// .bootstrapServers(BOOTSTRAP_SERVERS) +// .topic(TOPIC) +// .startupMode(StartupMode.SPECIFIC_OFFSETS) +// .specificOffsets(new HashMap<>())) +// .withFormat( +// new Json() +// .schema("") +// .failOnMissingField(true)) +// .toTableSource(); +// } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-json-schema.json b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-json-schema.json new file mode 100644 index 0000000000000..5167e5e872405 --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-json-schema.json @@ -0,0 +1,35 @@ +/* + * 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. + */ + +{ + "title": "Fruit", + "type": "object", + "properties": { + "name": { + "type": "string" + }, + "count": { + "type": "integer" + }, + "time": { + "description": "Age in years", + "type": "number" + } + }, + "required": ["name", "count", "time"] +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala index 534ef394f27bd..1e88d932ed6f9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala @@ -91,6 +91,11 @@ class TableSchema( } } + /** + * Returns the number of columns. + */ + def getColumnNum: Int = columnNames.length + /** * Returns all column names as an array. */ From db2c510fb4f171c9e9940759e5fbaf466ec74474 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 19 Feb 2018 13:35:45 +0100 Subject: [PATCH 005/268] [FLINK-8538] [table] Improve unified table sources This closes #5564. --- .../flink-connector-kafka-0.10/pom.xml | 8 + .../kafka/Kafka010JsonTableSourceFactory.java | 5 +- .../resources/tableSourceConverter.properties | 29 - ...> Kafka010JsonTableSourceFactoryTest.java} | 18 +- .../flink-connector-kafka-0.11/pom.xml | 8 + .../kafka/Kafka011JsonTableSourceFactory.java | 5 +- .../resources/tableSourceConverter.properties | 29 - ...> Kafka011JsonTableSourceFactoryTest.java} | 18 +- .../flink-connector-kafka-0.8/pom.xml | 8 + .../kafka/Kafka08JsonTableSourceFactory.java | 5 +- .../resources/tableSourceConverter.properties | 29 - ...=> Kafka08JsonTableSourceFactoryTest.java} | 19 +- .../flink-connector-kafka-0.9/pom.xml | 8 + .../kafka/Kafka09JsonTableSourceFactory.java | 5 +- .../resources/tableSourceConverter.properties | 29 - ...=> Kafka09JsonTableSourceFactoryTest.java} | 18 +- .../flink-connector-kafka-base/pom.xml | 16 +- .../kafka/KafkaJsonTableSource.java | 24 +- .../kafka/KafkaJsonTableSourceFactory.java | 267 +++--- .../connectors/kafka/KafkaTableSource.java | 42 +- .../apache/flink/table/descriptors/Kafka.java | 239 ++--- .../table/descriptors/KafkaValidator.java | 226 ++--- .../KafkaJsonTableFromDescriptorTestBase.java | 127 --- .../KafkaJsonTableSourceFactoryTestBase.java | 145 +++ .../flink/table/descriptors/KafkaTest.java | 113 +++ .../src/test/resources/kafka-json-schema.json | 35 - flink-dist/pom.xml | 16 - flink-formats/flink-json/pom.xml | 29 +- .../apache/flink/table/descriptors/Json.java | 129 +++ .../table/descriptors/JsonValidator.java | 55 ++ .../flink/table/descriptors/JsonTest.java | 124 +++ .../client/gateway/local/LocalExecutor.java | 2 +- flink-libraries/flink-table/pom.xml | 12 + .../resources/tableSourceConverter.properties | 7 + .../apache/flink/table/api/TableSchema.scala | 6 +- .../table/catalog/ExternalCatalogTable.scala | 19 +- .../catalog/ExternalTableSourceUtil.scala | 2 +- .../BatchTableSourceDescriptor.scala | 2 +- .../descriptors/ConnectorDescriptor.scala | 9 +- .../ConnectorDescriptorValidator.scala | 18 +- .../apache/flink/table/descriptors/Csv.scala | 13 +- .../descriptors/DescriptorProperties.scala | 833 +++++++++++++++--- .../flink/table/descriptors/FileSystem.scala | 5 +- .../table/descriptors/FormatDescriptor.scala | 4 +- .../FormatDescriptorValidator.scala | 23 +- .../apache/flink/table/descriptors/Json.scala | 78 -- .../table/descriptors/JsonValidator.scala | 41 - .../table/descriptors/MetadataValidator.scala | 6 +- .../flink/table/descriptors/Rowtime.scala | 8 +- .../table/descriptors/RowtimeValidator.scala | 179 ++-- .../flink/table/descriptors/Schema.scala | 14 +- .../table/descriptors/SchemaValidator.scala | 171 +++- .../flink/table/descriptors/Statistics.scala | 6 +- .../descriptors/StatisticsValidator.scala | 21 +- .../StreamTableSourceDescriptor.scala | 2 +- .../descriptors/TableSourceDescriptor.scala | 3 +- .../table/sources/CsvTableSourceFactory.scala | 43 +- .../table/sources/TableSourceFactory.scala | 11 +- .../sources/TableSourceFactoryService.scala | 26 +- .../tsextractors/StreamRecordTimestamp.scala | 5 +- .../flink/table/descriptors/CsvTest.scala | 83 +- .../descriptors/DescriptorTestBase.scala | 65 +- .../table/descriptors/FileSystemTest.scala | 33 +- .../flink/table/descriptors/JsonTest.scala | 77 -- .../table/descriptors/MetadataTest.scala | 38 +- .../flink/table/descriptors/RowtimeTest.scala | 59 +- .../flink/table/descriptors/SchemaTest.scala | 84 +- .../descriptors/SchemaValidatorTest.scala | 76 ++ .../table/descriptors/StatisticsTest.scala | 72 +- .../TableSourceFactoryServiceTest.scala | 20 +- .../sources/TestTableSourceFactory.scala | 8 +- 71 files changed, 2561 insertions(+), 1451 deletions(-) delete mode 100644 flink-connectors/flink-connector-kafka-0.10/src/main/resources/tableSourceConverter.properties rename flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka010TableSourceFactoryTest.java => Kafka010JsonTableSourceFactoryTest.java} (72%) delete mode 100644 flink-connectors/flink-connector-kafka-0.11/src/main/resources/tableSourceConverter.properties rename flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka011TableSourceFactoryTest.java => Kafka011JsonTableSourceFactoryTest.java} (72%) delete mode 100644 flink-connectors/flink-connector-kafka-0.8/src/main/resources/tableSourceConverter.properties rename flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka08TableSourceFactoryTest.java => Kafka08JsonTableSourceFactoryTest.java} (68%) delete mode 100644 flink-connectors/flink-connector-kafka-0.9/src/main/resources/tableSourceConverter.properties rename flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/{Kafka09TableSourceFactoryTest.java => Kafka09JsonTableSourceFactoryTest.java} (73%) delete mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableFromDescriptorTestBase.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java create mode 100644 flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/table/descriptors/KafkaTest.java delete mode 100644 flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-json-schema.json create mode 100644 flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java create mode 100644 flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java create mode 100644 flink-formats/flink-json/src/test/java/org/apache/flink/table/descriptors/JsonTest.java delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Json.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/JsonValidator.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/JsonTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala diff --git a/flink-connectors/flink-connector-kafka-0.10/pom.xml b/flink-connectors/flink-connector-kafka-0.10/pom.xml index 8b4ff38ab73f3..2d273aec46b16 100644 --- a/flink-connectors/flink-connector-kafka-0.10/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.10/pom.xml @@ -192,6 +192,14 @@ under the License. test + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + test-jar + test + + diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java index 1d03f6c4e8c03..c639a44289e64 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactory.java @@ -18,12 +18,13 @@ package org.apache.flink.streaming.connectors.kafka; -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_010; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_010; /** * Factory for creating configured instances of {@link Kafka010JsonTableSource}. */ public class Kafka010JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + @Override protected KafkaJsonTableSource.Builder createBuilder() { return new Kafka010JsonTableSource.Builder(); @@ -31,6 +32,6 @@ protected KafkaJsonTableSource.Builder createBuilder() { @Override protected String kafkaVersion() { - return KAFKA_VERSION_VALUE_010; + return CONNECTOR_VERSION_VALUE_010; } } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/tableSourceConverter.properties b/flink-connectors/flink-connector-kafka-0.10/src/main/resources/tableSourceConverter.properties deleted file mode 100644 index 5409b49703088..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/resources/tableSourceConverter.properties +++ /dev/null @@ -1,29 +0,0 @@ -# -# 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. -# - -################################################################################ -# The config file is used to specify the packages of current module where -# to find TableSourceConverter implementation class annotated with TableType. -# If there are multiple packages to scan, put those packages together into a -# string separated with ',', for example, org.package1,org.package2. -# Please notice: -# It's better to have a tableSourceConverter.properties in each connector Module -# which offers converters instead of put all information into the -# tableSourceConverter.properties of flink-table module. -################################################################################ -scan.packages=org.apache.flink.streaming.connectors.kafka diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactoryTest.java similarity index 72% rename from flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java rename to flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactoryTest.java index 15b89e8e2f85a..22cf659ffeb93 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSourceFactoryTest.java @@ -18,24 +18,20 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.table.descriptors.Kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_010; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_010; /** * Tests for {@link Kafka010JsonTableSourceFactory}. */ -public class Kafka010TableSourceFactoryTest extends KafkaJsonTableFromDescriptorTestBase { - protected String versionForTest() { - return KAFKA_VERSION_VALUE_010; - } +public class Kafka010JsonTableSourceFactoryTest extends KafkaJsonTableSourceFactoryTestBase { - protected KafkaJsonTableSource.Builder builderForTest() { - return Kafka010JsonTableSource.builder(); + @Override + protected String version() { + return CONNECTOR_VERSION_VALUE_010; } @Override - protected void extraSettings(KafkaTableSource.Builder builder, Kafka kafka) { - // no extra settings + protected KafkaJsonTableSource.Builder builder() { + return Kafka010JsonTableSource.builder(); } } diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml index 1e935f658a8a3..0fc1f134a9862 100644 --- a/flink-connectors/flink-connector-kafka-0.11/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml @@ -201,6 +201,14 @@ under the License. test + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + test-jar + test + + diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java index ca4d6ce01374d..6745bb294f12f 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactory.java @@ -18,12 +18,13 @@ package org.apache.flink.streaming.connectors.kafka; -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_011; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_011; /** * Factory for creating configured instances of {@link Kafka011JsonTableSource}. */ public class Kafka011JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + @Override protected KafkaJsonTableSource.Builder createBuilder() { return new Kafka011JsonTableSource.Builder(); @@ -31,6 +32,6 @@ protected KafkaJsonTableSource.Builder createBuilder() { @Override protected String kafkaVersion() { - return KAFKA_VERSION_VALUE_011; + return CONNECTOR_VERSION_VALUE_011; } } diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/tableSourceConverter.properties b/flink-connectors/flink-connector-kafka-0.11/src/main/resources/tableSourceConverter.properties deleted file mode 100644 index 5409b49703088..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/resources/tableSourceConverter.properties +++ /dev/null @@ -1,29 +0,0 @@ -# -# 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. -# - -################################################################################ -# The config file is used to specify the packages of current module where -# to find TableSourceConverter implementation class annotated with TableType. -# If there are multiple packages to scan, put those packages together into a -# string separated with ',', for example, org.package1,org.package2. -# Please notice: -# It's better to have a tableSourceConverter.properties in each connector Module -# which offers converters instead of put all information into the -# tableSourceConverter.properties of flink-table module. -################################################################################ -scan.packages=org.apache.flink.streaming.connectors.kafka diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactoryTest.java similarity index 72% rename from flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java rename to flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactoryTest.java index 84ac39b6f47e8..ed92863ba6d6f 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011JsonTableSourceFactoryTest.java @@ -18,24 +18,20 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.table.descriptors.Kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_011; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_011; /** * Tests for {@link Kafka011JsonTableSourceFactory}. */ -public class Kafka011TableSourceFactoryTest extends KafkaJsonTableFromDescriptorTestBase { - protected String versionForTest() { - return KAFKA_VERSION_VALUE_011; - } +public class Kafka011JsonTableSourceFactoryTest extends KafkaJsonTableSourceFactoryTestBase { - protected KafkaJsonTableSource.Builder builderForTest() { - return Kafka011JsonTableSource.builder(); + @Override + protected String version() { + return CONNECTOR_VERSION_VALUE_011; } @Override - protected void extraSettings(KafkaTableSource.Builder builder, Kafka kafka) { - // no extra settings + protected KafkaJsonTableSource.Builder builder() { + return Kafka011JsonTableSource.builder(); } } diff --git a/flink-connectors/flink-connector-kafka-0.8/pom.xml b/flink-connectors/flink-connector-kafka-0.8/pom.xml index a58591e5c6d80..43a9dacdc8f33 100644 --- a/flink-connectors/flink-connector-kafka-0.8/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.8/pom.xml @@ -204,6 +204,14 @@ under the License. test + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + test-jar + test + + diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java index e4e50960b627c..2da805a24b8ee 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactory.java @@ -18,12 +18,13 @@ package org.apache.flink.streaming.connectors.kafka; -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_08; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_08; /** * Factory for creating configured instances of {@link Kafka08JsonTableSource}. */ public class Kafka08JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + @Override protected KafkaJsonTableSource.Builder createBuilder() { return new Kafka08JsonTableSource.Builder(); @@ -31,6 +32,6 @@ protected KafkaJsonTableSource.Builder createBuilder() { @Override protected String kafkaVersion() { - return KAFKA_VERSION_VALUE_08; + return CONNECTOR_VERSION_VALUE_08; } } diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/tableSourceConverter.properties b/flink-connectors/flink-connector-kafka-0.8/src/main/resources/tableSourceConverter.properties deleted file mode 100644 index 5409b49703088..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/resources/tableSourceConverter.properties +++ /dev/null @@ -1,29 +0,0 @@ -# -# 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. -# - -################################################################################ -# The config file is used to specify the packages of current module where -# to find TableSourceConverter implementation class annotated with TableType. -# If there are multiple packages to scan, put those packages together into a -# string separated with ',', for example, org.package1,org.package2. -# Please notice: -# It's better to have a tableSourceConverter.properties in each connector Module -# which offers converters instead of put all information into the -# tableSourceConverter.properties of flink-table module. -################################################################################ -scan.packages=org.apache.flink.streaming.connectors.kafka diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactoryTest.java similarity index 68% rename from flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java rename to flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactoryTest.java index a2edc09ec125f..0238b2bf4f935 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceFactoryTest.java @@ -18,25 +18,20 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.table.descriptors.Kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_08; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_08; /** * Tests for {@link Kafka08JsonTableSourceFactory}. */ -public class Kafka08TableSourceFactoryTest extends KafkaJsonTableFromDescriptorTestBase { - protected String versionForTest() { - return KAFKA_VERSION_VALUE_08; - } +public class Kafka08JsonTableSourceFactoryTest extends KafkaJsonTableSourceFactoryTestBase { - protected KafkaJsonTableSource.Builder builderForTest() { - return Kafka08JsonTableSource.builder(); + @Override + protected String version() { + return CONNECTOR_VERSION_VALUE_08; } @Override - protected void extraSettings(KafkaTableSource.Builder builder, Kafka kafka) { - builder.getKafkaProps().put("zookeeper.connect", "localhost:1111"); - kafka.zookeeperConnect("localhost:1111"); + protected KafkaJsonTableSource.Builder builder() { + return Kafka08JsonTableSource.builder(); } } diff --git a/flink-connectors/flink-connector-kafka-0.9/pom.xml b/flink-connectors/flink-connector-kafka-0.9/pom.xml index d07cb5a83ea2a..1999a8d59c1d8 100644 --- a/flink-connectors/flink-connector-kafka-0.9/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.9/pom.xml @@ -173,6 +173,14 @@ under the License. test + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + test-jar + test + + org.apache.hadoop hadoop-minikdc diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java index bbda4ae66d636..9207426869e50 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactory.java @@ -18,12 +18,13 @@ package org.apache.flink.streaming.connectors.kafka; -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_09; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_09; /** * Factory for creating configured instances of {@link Kafka09JsonTableSource}. */ public class Kafka09JsonTableSourceFactory extends KafkaJsonTableSourceFactory { + @Override protected KafkaJsonTableSource.Builder createBuilder() { return new Kafka09JsonTableSource.Builder(); @@ -31,6 +32,6 @@ protected KafkaJsonTableSource.Builder createBuilder() { @Override protected String kafkaVersion() { - return KAFKA_VERSION_VALUE_09; + return CONNECTOR_VERSION_VALUE_09; } } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/tableSourceConverter.properties b/flink-connectors/flink-connector-kafka-0.9/src/main/resources/tableSourceConverter.properties deleted file mode 100644 index 5409b49703088..0000000000000 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/resources/tableSourceConverter.properties +++ /dev/null @@ -1,29 +0,0 @@ -# -# 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. -# - -################################################################################ -# The config file is used to specify the packages of current module where -# to find TableSourceConverter implementation class annotated with TableType. -# If there are multiple packages to scan, put those packages together into a -# string separated with ',', for example, org.package1,org.package2. -# Please notice: -# It's better to have a tableSourceConverter.properties in each connector Module -# which offers converters instead of put all information into the -# tableSourceConverter.properties of flink-table module. -################################################################################ -scan.packages=org.apache.flink.streaming.connectors.kafka diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactoryTest.java similarity index 73% rename from flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java rename to flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactoryTest.java index fc85ea77a07cd..dd545e9395138 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSourceFactoryTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceFactoryTest.java @@ -18,24 +18,20 @@ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.table.descriptors.Kafka; - -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION_VALUE_09; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_VERSION_VALUE_09; /** * Factory for creating configured instances of {@link Kafka09JsonTableSource}. */ -public class Kafka09TableSourceFactoryTest extends KafkaJsonTableFromDescriptorTestBase { - protected String versionForTest() { - return KAFKA_VERSION_VALUE_09; - } +public class Kafka09JsonTableSourceFactoryTest extends KafkaJsonTableSourceFactoryTestBase { - protected KafkaJsonTableSource.Builder builderForTest() { - return Kafka09JsonTableSource.builder(); + @Override + protected String version() { + return CONNECTOR_VERSION_VALUE_09; } @Override - protected void extraSettings(KafkaTableSource.Builder builder, Kafka kafka) { - // no extra settings + protected KafkaJsonTableSource.Builder builder() { + return Kafka09JsonTableSource.builder(); } } diff --git a/flink-connectors/flink-connector-kafka-base/pom.xml b/flink-connectors/flink-connector-kafka-base/pom.xml index 2ccaa2e5c1d3e..cea389604596d 100644 --- a/flink-connectors/flink-connector-kafka-base/pom.xml +++ b/flink-connectors/flink-connector-kafka-base/pom.xml @@ -198,24 +198,18 @@ under the License. test - - org.apache.hadoop - hadoop-minikdc - ${minikdc.version} - test - - org.apache.flink - flink-scala_${scala.binary.version} + flink-table_${scala.binary.version} ${project.version} + test-jar test - org.apache.flink - flink-streaming-scala_${scala.binary.version} - ${project.version} + org.apache.hadoop + hadoop-minikdc + ${minikdc.version} test diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java index d2dafe72b3495..b2bb8ff773bf4 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java @@ -84,23 +84,29 @@ protected JsonRowDeserializationSchema getDeserializationSchema() { @Override public String explainSource() { - return "KafkaJSONTableSource"; + return "KafkaJsonTableSource"; } @Override - public boolean equals(Object other) { - if (super.equals(other)) { - KafkaJsonTableSource otherSource = (KafkaJsonTableSource) other; - return Objects.equals(failOnMissingField, otherSource.failOnMissingField) - && Objects.equals(jsonSchema, otherSource.jsonSchema) - && Objects.equals(fieldMapping, otherSource.fieldMapping); + public boolean equals(Object o) { + if (this == o) { + return true; } - return false; + if (!(o instanceof KafkaJsonTableSource)) { + return false; + } + if (!super.equals(o)) { + return false; + } + KafkaJsonTableSource that = (KafkaJsonTableSource) o; + return failOnMissingField == that.failOnMissingField && + Objects.equals(jsonSchema, that.jsonSchema) && + Objects.equals(fieldMapping, that.fieldMapping); } @Override public int hashCode() { - return 31 * super.hashCode() + Objects.hash(failOnMissingField, jsonSchema, fieldMapping); + return Objects.hash(super.hashCode(), jsonSchema, fieldMapping, failOnMissingField); } //////// SETTERS FOR OPTIONAL PARAMETERS diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java index 918b83357ff56..28973149abd33 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactory.java @@ -21,65 +21,74 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.formats.json.JsonSchemaConverter; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.descriptors.JsonValidator; import org.apache.flink.table.descriptors.KafkaValidator; import org.apache.flink.table.descriptors.SchemaValidator; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.TableSource; import org.apache.flink.table.sources.TableSourceFactory; import org.apache.flink.types.Row; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_PROPERTY_VERSION; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE; -import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_VERSION; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; -import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA_STRING; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_JSON_SCHEMA; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA; import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_TYPE_VALUE; -import static org.apache.flink.table.descriptors.KafkaValidator.BOOTSTRAP_SERVERS; -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TYPE_VALUE; -import static org.apache.flink.table.descriptors.KafkaValidator.GROUP_ID; -import static org.apache.flink.table.descriptors.KafkaValidator.JSON_FIELD; -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION; -import static org.apache.flink.table.descriptors.KafkaValidator.OFFSET; -import static org.apache.flink.table.descriptors.KafkaValidator.PARTITION; -import static org.apache.flink.table.descriptors.KafkaValidator.SPECIFIC_OFFSETS; -import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE; -import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE_VALUE_EARLIEST; -import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE_VALUE_GROUP_OFFSETS; -import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE_VALUE_LATEST; -import static org.apache.flink.table.descriptors.KafkaValidator.STARTUP_MODE_VALUE_SPECIFIC_OFFSETS; -import static org.apache.flink.table.descriptors.KafkaValidator.TABLE_FIELD; -import static org.apache.flink.table.descriptors.KafkaValidator.TABLE_JSON_MAPPING; -import static org.apache.flink.table.descriptors.KafkaValidator.TOPIC; -import static org.apache.flink.table.descriptors.KafkaValidator.ZOOKEEPER_CONNECT; -import static org.apache.flink.table.descriptors.SchemaValidator.PROCTIME; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES_KEY; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES_VALUE; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_SPECIFIC_OFFSETS; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_SPECIFIC_OFFSETS_OFFSET; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_SPECIFIC_OFFSETS_PARTITION; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_STARTUP_MODE; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TOPIC; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TYPE_VALUE_KAFKA; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_CLASS; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_FROM; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_SERIALIZED; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_CLASS; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_DELAY; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_SERIALIZED; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; -import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_VERSION; - -import scala.Option; -import scala.collection.JavaConversions; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_FROM; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_PROCTIME; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; /** * Factory for creating configured instances of {@link KafkaJsonTableSource}. */ public abstract class KafkaJsonTableSourceFactory implements TableSourceFactory { + @Override public Map requiredContext() { Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE); // kafka connector - context.put(FORMAT_TYPE(), FORMAT_TYPE_VALUE()); // Json format - context.put(KAFKA_VERSION, kafkaVersion()); // for different implementations - context.put(CONNECTOR_VERSION(), "1"); - context.put(FORMAT_VERSION(), "1"); - context.put(SCHEMA_VERSION(), "1"); + context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE_KAFKA); // kafka + context.put(CONNECTOR_VERSION(), kafkaVersion()); + + context.put(FORMAT_TYPE(), FORMAT_TYPE_VALUE); // json format + + context.put(CONNECTOR_PROPERTY_VERSION(), "1"); // backwards compatibility + context.put(FORMAT_PROPERTY_VERSION(), "1"); + return context; } @@ -88,124 +97,137 @@ public List supportedProperties() { List properties = new ArrayList<>(); // kafka - properties.add(KAFKA_VERSION); - properties.add(BOOTSTRAP_SERVERS); - properties.add(GROUP_ID); - properties.add(ZOOKEEPER_CONNECT); - properties.add(TOPIC); - properties.add(STARTUP_MODE); - properties.add(SPECIFIC_OFFSETS + ".#." + PARTITION); - properties.add(SPECIFIC_OFFSETS + ".#." + OFFSET); + properties.add(CONNECTOR_TOPIC); + properties.add(CONNECTOR_PROPERTIES); + properties.add(CONNECTOR_PROPERTIES + ".#." + CONNECTOR_PROPERTIES_KEY); + properties.add(CONNECTOR_PROPERTIES + ".#." + CONNECTOR_PROPERTIES_VALUE); + properties.add(CONNECTOR_STARTUP_MODE); + properties.add(CONNECTOR_SPECIFIC_OFFSETS + ".#." + CONNECTOR_SPECIFIC_OFFSETS_PARTITION); + properties.add(CONNECTOR_SPECIFIC_OFFSETS + ".#." + CONNECTOR_SPECIFIC_OFFSETS_OFFSET); // json format - properties.add(FORMAT_SCHEMA_STRING()); - properties.add(FORMAT_FAIL_ON_MISSING_FIELD()); - - // table json mapping - properties.add(TABLE_JSON_MAPPING + ".#." + TABLE_FIELD); - properties.add(TABLE_JSON_MAPPING + ".#." + JSON_FIELD); + properties.add(FORMAT_JSON_SCHEMA); + properties.add(FORMAT_SCHEMA); + properties.add(FORMAT_FAIL_ON_MISSING_FIELD); + properties.add(FORMAT_DERIVE_SCHEMA()); // schema - properties.add(SCHEMA() + ".#." + DescriptorProperties.TYPE()); - properties.add(SCHEMA() + ".#." + DescriptorProperties.NAME()); + properties.add(SCHEMA() + ".#." + SCHEMA_TYPE()); + properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); + properties.add(SCHEMA() + ".#." + SCHEMA_FROM()); // time attributes - properties.add(SCHEMA() + ".#." + PROCTIME()); -// properties.add(SCHEMA() + ".#." + ROWTIME() + ".#." + TIMESTAMPS_CLASS()); -// properties.add(SCHEMA() + ".#." + ROWTIME() + ".#." + TIMESTAMPS_TYPE()); + properties.add(SCHEMA() + ".#." + SCHEMA_PROCTIME()); + properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_TYPE()); + properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_FROM()); + properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_CLASS()); + properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_SERIALIZED()); + properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_TYPE()); + properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_CLASS()); + properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_SERIALIZED()); + properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_DELAY()); return properties; } @Override public TableSource create(Map properties) { - DescriptorProperties params = new DescriptorProperties(true); + final DescriptorProperties params = new DescriptorProperties(true); params.putProperties(properties); // validate + new SchemaValidator(true).validate(params); new KafkaValidator().validate(params); new JsonValidator().validate(params); - new SchemaValidator(true).validate(params); // build - KafkaJsonTableSource.Builder builder = createBuilder(); - Properties kafkaProps = new Properties(); - - // Set the required parameters. - String topic = params.getString(TOPIC).get(); - TableSchema tableSchema = params.getTableSchema(SCHEMA()).get(); - - kafkaProps.put(BOOTSTRAP_SERVERS, params.getString(BOOTSTRAP_SERVERS).get()); - kafkaProps.put(GROUP_ID, params.getString(GROUP_ID).get()); - - // Set the zookeeper connect for kafka 0.8. - Option zkConnect = params.getString(ZOOKEEPER_CONNECT); - if (zkConnect.isDefined()) { - kafkaProps.put(ZOOKEEPER_CONNECT, zkConnect.get()); - } - - builder.withKafkaProperties(kafkaProps).forTopic(topic).withSchema(tableSchema); - - // Set the startup mode. - String startupMode = params.getString(STARTUP_MODE).get(); - if (null != startupMode) { - switch (startupMode) { - case STARTUP_MODE_VALUE_EARLIEST: + final KafkaJsonTableSource.Builder builder = createBuilder(); + + // topic + final String topic = params.getString(CONNECTOR_TOPIC); + builder.forTopic(topic); + + // properties + final Properties props = new Properties(); + final List> propsList = params.getFixedIndexedProperties( + CONNECTOR_PROPERTIES, + Arrays.asList(CONNECTOR_PROPERTIES_KEY, CONNECTOR_PROPERTIES_VALUE)); + propsList.forEach(kv -> props.put( + params.getString(kv.get(CONNECTOR_PROPERTIES_KEY)), + params.getString(kv.get(CONNECTOR_PROPERTIES_VALUE)) + )); + builder.withKafkaProperties(props); + + // startup mode + params + .getOptionalString(CONNECTOR_STARTUP_MODE) + .ifPresent(startupMode -> { + switch (startupMode) { + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_EARLIEST: builder.fromEarliest(); break; - case STARTUP_MODE_VALUE_LATEST: + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_LATEST: builder.fromLatest(); break; - case STARTUP_MODE_VALUE_GROUP_OFFSETS: + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_GROUP_OFFSETS: builder.fromGroupOffsets(); break; - case STARTUP_MODE_VALUE_SPECIFIC_OFFSETS: - Map partitions = JavaConversions. - mapAsJavaMap(params.getIndexedProperty(SPECIFIC_OFFSETS, PARTITION)); - Map offsetMap = new HashMap<>(); - for (int i = 0; i < partitions.size(); i++) { - offsetMap.put( - new KafkaTopicPartition( - topic, - Integer.valueOf(params.getString( - SPECIFIC_OFFSETS + "" + "." + i + "." + PARTITION).get())), - Long.valueOf(params.getString( - SPECIFIC_OFFSETS + "" + "." + i + "." + OFFSET).get())); - } + + case KafkaValidator.CONNECTOR_STARTUP_MODE_VALUE_SPECIFIC_OFFSETS: + final Map offsetMap = new HashMap<>(); + + final List> offsetList = params.getFixedIndexedProperties( + CONNECTOR_SPECIFIC_OFFSETS, + Arrays.asList(CONNECTOR_SPECIFIC_OFFSETS_PARTITION, CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); + offsetList.forEach(kv -> { + final int partition = params.getInt(kv.get(CONNECTOR_SPECIFIC_OFFSETS_PARTITION)); + final long offset = params.getLong(kv.get(CONNECTOR_SPECIFIC_OFFSETS_OFFSET)); + final KafkaTopicPartition topicPartition = new KafkaTopicPartition(topic, partition); + offsetMap.put(topicPartition, offset); + }); builder.fromSpecificOffsets(offsetMap); break; - } - } - - // Set whether fail on missing JSON field. - Option failOnMissing = params.getString(FORMAT_FAIL_ON_MISSING_FIELD()); - if (failOnMissing.isDefined()) { - builder.failOnMissingField(Boolean.valueOf(failOnMissing.get())); + } + }); + + // missing field + params.getOptionalBoolean(FORMAT_FAIL_ON_MISSING_FIELD).ifPresent(builder::failOnMissingField); + + // json schema + final TableSchema formatSchema; + if (params.containsKey(FORMAT_SCHEMA)) { + final TypeInformation info = params.getType(FORMAT_SCHEMA); + formatSchema = TableSchema.fromTypeInfo(info); + } else if (params.containsKey(FORMAT_JSON_SCHEMA)) { + final TypeInformation info = JsonSchemaConverter.convert(params.getString(FORMAT_JSON_SCHEMA)); + formatSchema = TableSchema.fromTypeInfo(info); + } else { + formatSchema = SchemaValidator.deriveFormatFields(params); } + builder.forJsonSchema(formatSchema); - // Set the JSON schema. - Option jsonSchema = params.getString(FORMAT_SCHEMA_STRING()); - if (jsonSchema.isDefined()) { - TypeInformation jsonSchemaType = JsonSchemaConverter.convert(jsonSchema.get()); - builder.forJsonSchema(TableSchema.fromTypeInfo(jsonSchemaType)); - } - - // Set the table => JSON fields mapping. - Map mappingTableFields = JavaConversions. - mapAsJavaMap(params.getIndexedProperty(TABLE_JSON_MAPPING, TABLE_FIELD)); - - if (!mappingTableFields.isEmpty()) { - Map tableJsonMapping = new HashMap<>(); - for (int i = 0; i < mappingTableFields.size(); i++) { - tableJsonMapping.put(params.getString(TABLE_JSON_MAPPING + "." + i + "." + TABLE_FIELD).get(), - params.getString(TABLE_JSON_MAPPING + "." + i + "." + JSON_FIELD).get() - ); - } - builder.withTableToJsonMapping(tableJsonMapping); + // schema + final TableSchema schema = params.getTableSchema(SCHEMA()); + builder.withSchema(schema); + + // proctime + SchemaValidator.deriveProctimeAttribute(params).ifPresent(builder::withProctimeAttribute); + + // rowtime + final List descriptors = SchemaValidator.deriveRowtimeAttributes(params); + if (descriptors.size() > 1) { + throw new TableException("More than one rowtime attribute is not supported yet."); + } else if (descriptors.size() == 1) { + final RowtimeAttributeDescriptor desc = descriptors.get(0); + builder.withRowtimeAttribute(desc.getAttributeName(), desc.getTimestampExtractor(), desc.getWatermarkStrategy()); } - // Set the time attributes. - setTimeAttributes(tableSchema, params, builder); + // field mapping + final Map mapping = SchemaValidator.deriveFieldMapping(params, Optional.of(formatSchema)); + builder.withTableToJsonMapping(mapping); return builder.build(); } @@ -213,15 +235,4 @@ public TableSource create(Map properties) { protected abstract KafkaJsonTableSource.Builder createBuilder(); protected abstract String kafkaVersion(); - - private void setTimeAttributes(TableSchema schema, DescriptorProperties params, KafkaJsonTableSource.Builder builder) { - // TODO to deal with rowtime fields - Option proctimeField; - for (int i = 0; i < schema.getColumnNum(); i++) { - proctimeField = params.getString(SCHEMA() + "." + i + "." + PROCTIME()); - if (proctimeField.isDefined()) { - builder.withProctimeAttribute(schema.getColumnName(i).get()); - } - } - } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index 9ce3b8ed5a17c..134c483a95016 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -141,31 +141,27 @@ public String explainSource() { @Override public boolean equals(Object o) { - if (!o.getClass().equals(this.getClass())) { + if (this == o) { + return true; + } + if (!(o instanceof KafkaTableSource)) { return false; } - KafkaTableSource other = (KafkaTableSource) o; - return Objects.equals(topic, other.topic) - && Objects.equals(schema, other.schema) - && Objects.equals(properties, other.properties) - && Objects.equals(proctimeAttribute, other.proctimeAttribute) - && Objects.equals(returnType, other.returnType) - && Objects.equals(rowtimeAttributeDescriptors, other.rowtimeAttributeDescriptors) - && Objects.equals(specificStartupOffsets, other.specificStartupOffsets) - && Objects.equals(startupMode, other.startupMode); + KafkaTableSource that = (KafkaTableSource) o; + return Objects.equals(schema, that.schema) && + Objects.equals(topic, that.topic) && + Objects.equals(properties, that.properties) && + Objects.equals(returnType, that.returnType) && + Objects.equals(proctimeAttribute, that.proctimeAttribute) && + Objects.equals(rowtimeAttributeDescriptors, that.rowtimeAttributeDescriptors) && + startupMode == that.startupMode && + Objects.equals(specificStartupOffsets, that.specificStartupOffsets); } @Override public int hashCode() { - return Objects.hash( - topic, - schema, - properties, - proctimeAttribute, - returnType, - rowtimeAttributeDescriptors, - specificStartupOffsets, - startupMode); + return Objects.hash(schema, topic, properties, returnType, + proctimeAttribute, rowtimeAttributeDescriptors, startupMode, specificStartupOffsets); } /** @@ -211,9 +207,9 @@ protected void setProctimeAttribute(String proctimeAttribute) { // validate that field exists and is of correct type Option> tpe = schema.getType(proctimeAttribute); if (tpe.isEmpty()) { - throw new ValidationException("Processing time attribute " + proctimeAttribute + " is not present in TableSchema."); + throw new ValidationException("Processing time attribute '" + proctimeAttribute + "' is not present in TableSchema."); } else if (tpe.get() != Types.SQL_TIMESTAMP()) { - throw new ValidationException("Processing time attribute " + proctimeAttribute + " is not of type SQL_TIMESTAMP."); + throw new ValidationException("Processing time attribute '" + proctimeAttribute + "' is not of type SQL_TIMESTAMP."); } } this.proctimeAttribute = proctimeAttribute; @@ -230,9 +226,9 @@ protected void setRowtimeAttributeDescriptors(List r String rowtimeAttribute = desc.getAttributeName(); Option> tpe = schema.getType(rowtimeAttribute); if (tpe.isEmpty()) { - throw new ValidationException("Rowtime attribute " + rowtimeAttribute + " is not present in TableSchema."); + throw new ValidationException("Rowtime attribute '" + rowtimeAttribute + "' is not present in TableSchema."); } else if (tpe.get() != Types.SQL_TIMESTAMP()) { - throw new ValidationException("Rowtime attribute " + rowtimeAttribute + " is not of type SQL_TIMESTAMP."); + throw new ValidationException("Rowtime attribute '" + rowtimeAttribute + "' is not of type SQL_TIMESTAMP."); } } this.rowtimeAttributeDescriptors = rowtimeAttributeDescriptors; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java index 4733f6e002db8..45359587c1cd0 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/Kafka.java @@ -18,182 +18,199 @@ package org.apache.flink.table.descriptors; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; - -import static org.apache.flink.table.descriptors.KafkaValidator.BOOTSTRAP_SERVERS; -import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TYPE_VALUE; -import static org.apache.flink.table.descriptors.KafkaValidator.GROUP_ID; -import static org.apache.flink.table.descriptors.KafkaValidator.JSON_FIELD; -import static org.apache.flink.table.descriptors.KafkaValidator.KAFKA_VERSION; -import static org.apache.flink.table.descriptors.KafkaValidator.OFFSET; -import static org.apache.flink.table.descriptors.KafkaValidator.PARTITION; -import static org.apache.flink.table.descriptors.KafkaValidator.SPECIFIC_OFFSETS; -import static org.apache.flink.table.descriptors.KafkaValidator.TABLE_FIELD; -import static org.apache.flink.table.descriptors.KafkaValidator.TABLE_JSON_MAPPING; -import static org.apache.flink.table.descriptors.KafkaValidator.TOPIC; -import static org.apache.flink.table.descriptors.KafkaValidator.ZOOKEEPER_CONNECT; +import org.apache.flink.util.Preconditions; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; - -import scala.collection.JavaConversions; -import scala.collection.Seq; +import java.util.Properties; +import java.util.stream.Collectors; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES_KEY; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_PROPERTIES_VALUE; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_SPECIFIC_OFFSETS; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_SPECIFIC_OFFSETS_OFFSET; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_SPECIFIC_OFFSETS_PARTITION; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_STARTUP_MODE; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TOPIC; +import static org.apache.flink.table.descriptors.KafkaValidator.CONNECTOR_TYPE_VALUE_KAFKA; /** - * Connector descriptor for the kafka message queue. + * Connector descriptor for the Apache Kafka message queue. */ public class Kafka extends ConnectorDescriptor { - private Optional version = Optional.empty(); - private Optional bootstrapServers = Optional.empty(); - private Optional groupId = Optional.empty(); - private Optional topic = Optional.empty(); - private Optional zookeeperConnect = Optional.empty(); - private Optional> tableJsonMapping = Optional.empty(); - - private Optional startupMode = Optional.empty(); - private Optional> specificOffsets = Optional.empty(); + private String version; + private String topic; + private StartupMode startupMode; + private Map specificOffsets; + private Map kafkaProperties; + /** + * Connector descriptor for the Apache Kafka message queue. + */ public Kafka() { - super(CONNECTOR_TYPE_VALUE, 1); + super(CONNECTOR_TYPE_VALUE_KAFKA, 1, true); } /** - * Sets the kafka version. + * Sets the Kafka version to be used. * - * @param version - * Could be {@link KafkaValidator#KAFKA_VERSION_VALUE_011}, - * {@link KafkaValidator#KAFKA_VERSION_VALUE_010}, - * {@link KafkaValidator#KAFKA_VERSION_VALUE_09}, - * or {@link KafkaValidator#KAFKA_VERSION_VALUE_08}. + * @param version Kafka version. E.g., "0.8", "0.11", etc. */ public Kafka version(String version) { - this.version = Optional.of(version); + Preconditions.checkNotNull(version); + this.version = version; return this; } /** - * Sets the bootstrap servers for kafka. + * Sets the topic from which the table is read. + * + * @param topic The topic from which the table is read. */ - public Kafka bootstrapServers(String bootstrapServers) { - this.bootstrapServers = Optional.of(bootstrapServers); + public Kafka topic(String topic) { + Preconditions.checkNotNull(topic); + this.topic = topic; return this; } /** - * Sets the consumer group id. + * Sets the configuration properties for the Kafka consumer. Resets previously set properties. + * + * @param properties The configuration properties for the Kafka consumer. */ - public Kafka groupId(String groupId) { - this.groupId = Optional.of(groupId); + public Kafka properties(Properties properties) { + Preconditions.checkNotNull(properties); + if (this.kafkaProperties == null) { + this.kafkaProperties = new HashMap<>(); + } + this.kafkaProperties.clear(); + properties.forEach((k, v) -> this.kafkaProperties.put((String) k, (String) v)); return this; } /** - * Sets the topic to consume. + * Adds a configuration properties for the Kafka consumer. + * + * @param key property key for the Kafka consumer + * @param value property value for the Kafka consumer */ - public Kafka topic(String topic) { - this.topic = Optional.of(topic); + public Kafka property(String key, String value) { + Preconditions.checkNotNull(key); + Preconditions.checkNotNull(value); + if (this.kafkaProperties == null) { + this.kafkaProperties = new HashMap<>(); + } + kafkaProperties.put(key, value); return this; } /** - * Sets the startup mode. + * Configures to start reading from the earliest offset for all partitions. + * + * @see FlinkKafkaConsumerBase#setStartFromEarliest() */ - public Kafka startupMode(StartupMode startupMode) { - this.startupMode = Optional.of(startupMode); + public Kafka startFromEarliest() { + this.startupMode = StartupMode.EARLIEST; + this.specificOffsets = null; return this; } /** - * Sets the zookeeper hosts. Only required by kafka 0.8. + * Configures to start reading from the latest offset for all partitions. + * + * @see FlinkKafkaConsumerBase#setStartFromLatest() */ - public Kafka zookeeperConnect(String zookeeperConnect) { - this.zookeeperConnect = Optional.of(zookeeperConnect); + public Kafka startFromLatest() { + this.startupMode = StartupMode.LATEST; + this.specificOffsets = null; return this; } /** - * Sets the consume offsets for the topic set with {@link Kafka#topic(String)}. - * Only works in {@link StartupMode#SPECIFIC_OFFSETS} mode. + * Configures to start reading from any committed group offsets found in Zookeeper / Kafka brokers. + * + * @see FlinkKafkaConsumerBase#setStartFromGroupOffsets() */ - public Kafka specificOffsets(Map specificOffsets) { - this.specificOffsets = Optional.of(specificOffsets); + public Kafka startFromGroupOffsets() { + this.startupMode = StartupMode.GROUP_OFFSETS; + this.specificOffsets = null; return this; } /** - * Sets the mapping from logical table schema to json schema. + * Configures to start reading partitions from specific offsets, set independently for each partition. + * Resets previously set offsets. + * + * @param specificOffsets the specified offsets for partitions + * @see FlinkKafkaConsumerBase#setStartFromSpecificOffsets(Map) */ - public Kafka tableJsonMapping(Map jsonTableMapping) { - this.tableJsonMapping = Optional.of(jsonTableMapping); + public Kafka startFromSpecificOffsets(Map specificOffsets) { + this.startupMode = StartupMode.SPECIFIC_OFFSETS; + this.specificOffsets = Preconditions.checkNotNull(specificOffsets); return this; } + /** + * Configures to start reading partitions from specific offsets and specifies the given offset for + * the given partition. + * + * @param partition partition index + * @param specificOffset partition offset to start reading from + * @see FlinkKafkaConsumerBase#setStartFromSpecificOffsets(Map) + */ + public Kafka startFromSpecificOffset(int partition, long specificOffset) { + this.startupMode = StartupMode.SPECIFIC_OFFSETS; + if (this.specificOffsets == null) { + this.specificOffsets = new HashMap<>(); + } + this.specificOffsets.put(partition, specificOffset); + return this; + } + + /** + * Internal method for connector properties conversion. + */ @Override public void addConnectorProperties(DescriptorProperties properties) { - if (version.isPresent()) { - properties.putString(KAFKA_VERSION, version.get()); - } - if (bootstrapServers.isPresent()) { - properties.putString(BOOTSTRAP_SERVERS, bootstrapServers.get()); - } - if (groupId.isPresent()) { - properties.putString(GROUP_ID, groupId.get()); - } - if (topic.isPresent()) { - properties.putString(TOPIC, topic.get()); + if (version != null) { + properties.putString(CONNECTOR_VERSION(), version); } - if (zookeeperConnect.isPresent()) { - properties.putString(ZOOKEEPER_CONNECT, zookeeperConnect.get()); + + if (topic != null) { + properties.putString(CONNECTOR_TOPIC, topic); } - if (startupMode.isPresent()) { - Map map = KafkaValidator.normalizeStartupMode(startupMode.get()); - for (Map.Entry entry : map.entrySet()) { - properties.putString(entry.getKey(), entry.getValue()); - } + + if (startupMode != null) { + properties.putString(CONNECTOR_STARTUP_MODE, KafkaValidator.normalizeStartupMode(startupMode)); } - if (specificOffsets.isPresent()) { - List propertyKeys = new ArrayList<>(); - propertyKeys.add(PARTITION); - propertyKeys.add(OFFSET); - - List> propertyValues = new ArrayList<>(specificOffsets.get().size()); - for (Map.Entry entry : specificOffsets.get().entrySet()) { - List partitionOffset = new ArrayList<>(2); - partitionOffset.add(entry.getKey().toString()); - partitionOffset.add(entry.getValue().toString()); - propertyValues.add(JavaConversions.asScalaBuffer(partitionOffset).toSeq()); + + if (specificOffsets != null) { + final List> values = new ArrayList<>(); + for (Map.Entry specificOffset : specificOffsets.entrySet()) { + values.add(Arrays.asList(specificOffset.getKey().toString(), specificOffset.getValue().toString())); } properties.putIndexedFixedProperties( - SPECIFIC_OFFSETS, - JavaConversions.asScalaBuffer(propertyKeys).toSeq(), - JavaConversions.asScalaBuffer(propertyValues).toSeq() - ); + CONNECTOR_SPECIFIC_OFFSETS, + Arrays.asList(CONNECTOR_SPECIFIC_OFFSETS_PARTITION, CONNECTOR_SPECIFIC_OFFSETS_OFFSET), + values); } - if (tableJsonMapping.isPresent()) { - List propertyKeys = new ArrayList<>(); - propertyKeys.add(TABLE_FIELD); - propertyKeys.add(JSON_FIELD); - - List> mappingFields = new ArrayList<>(tableJsonMapping.get().size()); - for (Map.Entry entry : tableJsonMapping.get().entrySet()) { - List singleMapping = new ArrayList<>(2); - singleMapping.add(entry.getKey()); - singleMapping.add(entry.getValue()); - mappingFields.add(JavaConversions.asScalaBuffer(singleMapping).toSeq()); - } + + if (kafkaProperties != null) { properties.putIndexedFixedProperties( - TABLE_JSON_MAPPING, - JavaConversions.asScalaBuffer(propertyKeys).toSeq(), - JavaConversions.asScalaBuffer(mappingFields).toSeq() - ); + CONNECTOR_PROPERTIES, + Arrays.asList(CONNECTOR_PROPERTIES_KEY, CONNECTOR_PROPERTIES_VALUE), + this.kafkaProperties.entrySet().stream() + .map(e -> Arrays.asList(e.getKey(), e.getValue())) + .collect(Collectors.toList()) + ); } } - - @Override - public boolean needsFormat() { - return true; - } } diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java index a3ca22f90b912..3adc7c518a447 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/table/descriptors/KafkaValidator.java @@ -19,175 +19,97 @@ package org.apache.flink.table.descriptors; import org.apache.flink.streaming.connectors.kafka.config.StartupMode; -import org.apache.flink.table.api.ValidationException; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; - -import scala.Function0; -import scala.Tuple2; -import scala.collection.JavaConversions; -import scala.runtime.AbstractFunction0; -import scala.runtime.BoxedUnit; - +import java.util.function.Consumer; /** * The validator for {@link Kafka}. */ public class KafkaValidator extends ConnectorDescriptorValidator { - // fields - public static final String CONNECTOR_TYPE_VALUE = "kafka"; - public static final String KAFKA_VERSION = "kafka.version"; - public static final String BOOTSTRAP_SERVERS = "bootstrap.servers"; - public static final String GROUP_ID = "group.id"; - public static final String TOPIC = "topic"; - public static final String STARTUP_MODE = "startup.mode"; - public static final String SPECIFIC_OFFSETS = "specific.offsets"; - public static final String TABLE_JSON_MAPPING = "table.json.mapping"; - - public static final String PARTITION = "partition"; - public static final String OFFSET = "offset"; - - public static final String TABLE_FIELD = "table.field"; - public static final String JSON_FIELD = "json.field"; - - public static final String ZOOKEEPER_CONNECT = "zookeeper.connect"; // only required for 0.8 - // values - public static final String KAFKA_VERSION_VALUE_08 = "0.8"; - public static final String KAFKA_VERSION_VALUE_09 = "0.9"; - public static final String KAFKA_VERSION_VALUE_010 = "0.10"; - public static final String KAFKA_VERSION_VALUE_011 = "0.11"; - - public static final String STARTUP_MODE_VALUE_EARLIEST = "earliest-offset"; - public static final String STARTUP_MODE_VALUE_LATEST = "latest-offset"; - public static final String STARTUP_MODE_VALUE_GROUP_OFFSETS = "group-offsets"; - public static final String STARTUP_MODE_VALUE_SPECIFIC_OFFSETS = "specific-offsets"; - - // utils - public static Map normalizeStartupMode(StartupMode startupMode) { - Map mapPair = new HashMap<>(); - switch (startupMode) { - case EARLIEST: - mapPair.put(STARTUP_MODE, STARTUP_MODE_VALUE_EARLIEST); - break; - case LATEST: - mapPair.put(STARTUP_MODE, STARTUP_MODE_VALUE_LATEST); - break; - case GROUP_OFFSETS: - mapPair.put(STARTUP_MODE, STARTUP_MODE_VALUE_GROUP_OFFSETS); - break; - case SPECIFIC_OFFSETS: - mapPair.put(STARTUP_MODE, STARTUP_MODE_VALUE_SPECIFIC_OFFSETS); - break; - } - return mapPair; - } + public static final String CONNECTOR_TYPE_VALUE_KAFKA = "kafka"; + public static final String CONNECTOR_VERSION_VALUE_08 = "0.8"; + public static final String CONNECTOR_VERSION_VALUE_09 = "0.9"; + public static final String CONNECTOR_VERSION_VALUE_010 = "0.10"; + public static final String CONNECTOR_VERSION_VALUE_011 = "0.11"; + public static final String CONNECTOR_TOPIC = "connector.topic"; + public static final String CONNECTOR_STARTUP_MODE = "connector.startup-mode"; + public static final String CONNECTOR_STARTUP_MODE_VALUE_EARLIEST = "earliest-offset"; + public static final String CONNECTOR_STARTUP_MODE_VALUE_LATEST = "latest-offset"; + public static final String CONNECTOR_STARTUP_MODE_VALUE_GROUP_OFFSETS = "group-offsets"; + public static final String CONNECTOR_STARTUP_MODE_VALUE_SPECIFIC_OFFSETS = "specific-offsets"; + public static final String CONNECTOR_SPECIFIC_OFFSETS = "connector.specific-offsets"; + public static final String CONNECTOR_SPECIFIC_OFFSETS_PARTITION = "partition"; + public static final String CONNECTOR_SPECIFIC_OFFSETS_OFFSET = "offset"; + public static final String CONNECTOR_PROPERTIES = "connector.properties"; + public static final String CONNECTOR_PROPERTIES_KEY = "key"; + public static final String CONNECTOR_PROPERTIES_VALUE = "value"; @Override public void validate(DescriptorProperties properties) { super.validate(properties); - - AbstractFunction0 emptyValidator = new AbstractFunction0() { - @Override - public BoxedUnit apply() { - return BoxedUnit.UNIT; - } - }; - - properties.validateValue(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE, false); - - AbstractFunction0 version08Validator = new AbstractFunction0() { - @Override - public BoxedUnit apply() { - properties.validateString(ZOOKEEPER_CONNECT, false, 0, Integer.MAX_VALUE); - return BoxedUnit.UNIT; - } - }; - - Map> versionValidatorMap = new HashMap<>(); - versionValidatorMap.put(KAFKA_VERSION_VALUE_08, version08Validator); - versionValidatorMap.put(KAFKA_VERSION_VALUE_09, emptyValidator); - versionValidatorMap.put(KAFKA_VERSION_VALUE_010, emptyValidator); - versionValidatorMap.put(KAFKA_VERSION_VALUE_011, emptyValidator); - properties.validateEnum( - KAFKA_VERSION, + properties.validateValue(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE_KAFKA, false); + + final List versions = Arrays.asList( + CONNECTOR_VERSION_VALUE_08, + CONNECTOR_VERSION_VALUE_09, + CONNECTOR_VERSION_VALUE_010, + CONNECTOR_VERSION_VALUE_011); + properties.validateEnumValues(CONNECTOR_VERSION(), false, versions); + properties.validateString(CONNECTOR_TOPIC, false, 1, Integer.MAX_VALUE); + + final Map> specificOffsetValidators = new HashMap<>(); + specificOffsetValidators.put( + CONNECTOR_SPECIFIC_OFFSETS_PARTITION, + (prefix) -> properties.validateInt( + prefix + CONNECTOR_SPECIFIC_OFFSETS_PARTITION, false, - toScalaImmutableMap(versionValidatorMap) - ); - - properties.validateString(BOOTSTRAP_SERVERS, false, 1, Integer.MAX_VALUE); - properties.validateString(GROUP_ID, false, 1, Integer.MAX_VALUE); - properties.validateString(TOPIC, false, 1, Integer.MAX_VALUE); - - AbstractFunction0 specificOffsetsValidator = new AbstractFunction0() { - @Override - public BoxedUnit apply() { - Map partitions = JavaConversions.mapAsJavaMap( - properties.getIndexedProperty(SPECIFIC_OFFSETS, PARTITION)); - - Map offsets = JavaConversions.mapAsJavaMap( - properties.getIndexedProperty(SPECIFIC_OFFSETS, OFFSET)); - if (partitions.isEmpty() || offsets.isEmpty()) { - throw new ValidationException("Offsets must be set for SPECIFIC_OFFSETS mode."); - } - for (int i = 0; i < partitions.size(); ++i) { - properties.validateInt( - SPECIFIC_OFFSETS + "." + i + "." + PARTITION, - false, - 0, - Integer.MAX_VALUE); - properties.validateLong( - SPECIFIC_OFFSETS + "." + i + "." + OFFSET, - false, - 0, - Long.MAX_VALUE); - } - return BoxedUnit.UNIT; - } - }; - Map> startupModeValidatorMap = new HashMap<>(); - startupModeValidatorMap.put(STARTUP_MODE_VALUE_GROUP_OFFSETS, emptyValidator); - startupModeValidatorMap.put(STARTUP_MODE_VALUE_EARLIEST, emptyValidator); - startupModeValidatorMap.put(STARTUP_MODE_VALUE_LATEST, emptyValidator); - startupModeValidatorMap.put(STARTUP_MODE_VALUE_SPECIFIC_OFFSETS, specificOffsetsValidator); - - properties.validateEnum(STARTUP_MODE, true, toScalaImmutableMap(startupModeValidatorMap)); - validateTableJsonMapping(properties); + 0, + Integer.MAX_VALUE)); + specificOffsetValidators.put( + CONNECTOR_SPECIFIC_OFFSETS_OFFSET, + (prefix) -> properties.validateLong( + prefix + CONNECTOR_SPECIFIC_OFFSETS_OFFSET, + false, + 0, + Long.MAX_VALUE)); + + final Map> startupModeValidation = new HashMap<>(); + startupModeValidation.put(CONNECTOR_STARTUP_MODE_VALUE_GROUP_OFFSETS, properties.noValidation()); + startupModeValidation.put(CONNECTOR_STARTUP_MODE_VALUE_EARLIEST, properties.noValidation()); + startupModeValidation.put(CONNECTOR_STARTUP_MODE_VALUE_LATEST, properties.noValidation()); + startupModeValidation.put( + CONNECTOR_STARTUP_MODE_VALUE_SPECIFIC_OFFSETS, + prefix -> properties.validateFixedIndexedProperties(CONNECTOR_SPECIFIC_OFFSETS, false, specificOffsetValidators)); + properties.validateEnum(CONNECTOR_STARTUP_MODE, true, startupModeValidation); + + final Map> propertyValidators = new HashMap<>(); + propertyValidators.put( + CONNECTOR_PROPERTIES_KEY, + prefix -> properties.validateString(prefix + CONNECTOR_PROPERTIES_KEY, false, 1, Integer.MAX_VALUE)); + propertyValidators.put( + CONNECTOR_PROPERTIES_VALUE, + prefix -> properties.validateString(prefix + CONNECTOR_PROPERTIES_VALUE, false, 0, Integer.MAX_VALUE)); + properties.validateFixedIndexedProperties(CONNECTOR_PROPERTIES, true, propertyValidators); } - private void validateTableJsonMapping(DescriptorProperties properties) { - Map mappingTableField = JavaConversions.mapAsJavaMap( - properties.getIndexedProperty(TABLE_JSON_MAPPING, TABLE_FIELD)); - Map mappingJsonField = JavaConversions.mapAsJavaMap( - properties.getIndexedProperty(TABLE_JSON_MAPPING, JSON_FIELD)); - - if (mappingJsonField.size() != mappingJsonField.size()) { - throw new ValidationException("Table JSON mapping must be one to one."); - } - - for (int i = 0; i < mappingTableField.size(); i++) { - properties.validateString( - TABLE_JSON_MAPPING + "." + i + "." + TABLE_FIELD, - false, - 1, - Integer.MAX_VALUE); - properties.validateString( - TABLE_JSON_MAPPING + "." + i + "." + JSON_FIELD, - false, - 1, - Integer.MAX_VALUE); - } - } + // utilities - @SuppressWarnings("unchecked") - private scala.collection.immutable.Map toScalaImmutableMap(Map javaMap) { - final java.util.List> list = new java.util.ArrayList<>(javaMap.size()); - for (final java.util.Map.Entry entry : javaMap.entrySet()) { - list.add(scala.Tuple2.apply(entry.getKey(), entry.getValue())); + public static String normalizeStartupMode(StartupMode startupMode) { + switch (startupMode) { + case EARLIEST: + return CONNECTOR_STARTUP_MODE_VALUE_EARLIEST; + case LATEST: + return CONNECTOR_STARTUP_MODE_VALUE_LATEST; + case GROUP_OFFSETS: + return CONNECTOR_STARTUP_MODE_VALUE_GROUP_OFFSETS; + case SPECIFIC_OFFSETS: + return CONNECTOR_STARTUP_MODE_VALUE_SPECIFIC_OFFSETS; } - final scala.collection.Seq> seq = - scala.collection.JavaConverters.asScalaBufferConverter(list).asScala().toSeq(); - return (scala.collection.immutable.Map) scala.collection.immutable.Map$.MODULE$.apply(seq); + throw new IllegalArgumentException("Invalid startup mode."); } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableFromDescriptorTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableFromDescriptorTestBase.java deleted file mode 100644 index 964a62425481e..0000000000000 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableFromDescriptorTestBase.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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.streaming.connectors.kafka; - -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.java.StreamTableEnvironment; -import org.apache.flink.table.descriptors.Kafka; - -import org.mockito.Mockito; - -/** - * Tests for {@link KafkaJsonTableSourceFactory}. - */ -public abstract class KafkaJsonTableFromDescriptorTestBase { - private static final String GROUP_ID = "test-group"; - private static final String BOOTSTRAP_SERVERS = "localhost:1234"; - private static final String TOPIC = "test-topic"; - - protected abstract String versionForTest(); - - protected abstract KafkaJsonTableSource.Builder builderForTest(); - - protected abstract void extraSettings(KafkaTableSource.Builder builder, Kafka kafka); - - private static StreamExecutionEnvironment env = Mockito.mock(StreamExecutionEnvironment.class); - private static StreamTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env); - -// @Test -// public void buildJsonTableSourceTest() throws Exception { -// final URL url = getClass().getClassLoader().getResource("kafka-json-schema.json"); -// Objects.requireNonNull(url); -// final String schema = FileUtils.readFileUtf8(new File(url.getFile())); -// -// Map tableJsonMapping = new HashMap<>(); -// tableJsonMapping.put("fruit-name", "name"); -// tableJsonMapping.put("fruit-count", "count"); -// tableJsonMapping.put("event-time", "time"); -// -// // Construct with the builder. -// Properties props = new Properties(); -// props.put("group.id", GROUP_ID); -// props.put("bootstrap.servers", BOOTSTRAP_SERVERS); -// -// Map specificOffsets = new HashMap<>(); -// specificOffsets.put(new KafkaTopicPartition(TOPIC, 0), 100L); -// specificOffsets.put(new KafkaTopicPartition(TOPIC, 1), 123L); -// -// KafkaTableSource.Builder builder = builderForTest() -// .forJsonSchema(TableSchema.fromTypeInfo(JsonSchemaConverter.convert(schema))) -// .failOnMissingField(true) -// .withTableToJsonMapping(tableJsonMapping) -// .withKafkaProperties(props) -// .forTopic(TOPIC) -// .fromSpecificOffsets(specificOffsets) -// .withSchema( -// TableSchema.builder() -// .field("fruit-name", Types.STRING) -// .field("fruit-count", Types.INT) -// .field("event-time", Types.LONG) -// .field("proc-time", Types.SQL_TIMESTAMP) -// .build()) -// .withProctimeAttribute("proc-time"); -// -// // Construct with the descriptor. -// Map offsets = new HashMap<>(); -// offsets.put(0, 100L); -// offsets.put(1, 123L); -// Kafka kafka = new Kafka() -// .version(versionForTest()) -// .groupId(GROUP_ID) -// .bootstrapServers(BOOTSTRAP_SERVERS) -// .topic(TOPIC) -// .startupMode(StartupMode.SPECIFIC_OFFSETS) -// .specificOffsets(offsets) -// .tableJsonMapping(tableJsonMapping); -// extraSettings(builder, kafka); -// -// TableSource source = tEnv -// .from(kafka) -// .withFormat( -// new Json() -// .schema(schema) -// .failOnMissingField(true)) -// .withSchema(new Schema() -// .field("fruit-name", Types.STRING) -// .field("fruit-count", Types.INT) -// .field("event-time", Types.LONG) -// .field("proc-time", Types.SQL_TIMESTAMP).proctime()) -// .toTableSource(); -// -// Assert.assertEquals(builder.build(), source); -// } - -// @Test(expected = TableException.class) -// public void buildJsonTableSourceFailTest() { -// tEnv.from( -// new Kafka() -// .version(versionForTest()) -// .groupId(GROUP_ID) -// .bootstrapServers(BOOTSTRAP_SERVERS) -// .topic(TOPIC) -// .startupMode(StartupMode.SPECIFIC_OFFSETS) -// .specificOffsets(new HashMap<>())) -// .withFormat( -// new Json() -// .schema("") -// .failOnMissingField(true)) -// .toTableSource(); -// } -} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java new file mode 100644 index 0000000000000..2b081a9f9157d --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java @@ -0,0 +1,145 @@ +/* + * 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.streaming.connectors.kafka; + +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.formats.json.JsonSchemaConverter; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.descriptors.FormatDescriptor; +import org.apache.flink.table.descriptors.Json; +import org.apache.flink.table.descriptors.Kafka; +import org.apache.flink.table.descriptors.Schema; +import org.apache.flink.table.descriptors.TestTableSourceDescriptor; +import org.apache.flink.table.sources.TableSource; +import org.apache.flink.table.sources.TableSourceFactoryService; + +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for {@link KafkaJsonTableSourceFactory}. + */ +public abstract class KafkaJsonTableSourceFactoryTestBase { + + private static final String JSON_SCHEMA = + "{" + + " 'title': 'Fruit'," + + " 'type': 'object'," + + " 'properties': {" + + " 'name': {" + + " 'type': 'string'" + + " }," + + " 'count': {" + + " 'type': 'integer'" + + " }," + + " 'time': {" + + " 'description': 'Age in years'," + + " 'type': 'number'" + + " }" + " }," + + " 'required': ['name', 'count', 'time']" + + "}"; + + private static final String TOPIC = "test-topic"; + + protected abstract String version(); + + protected abstract KafkaJsonTableSource.Builder builder(); + + @Test + public void testTableSourceFromJsonSchema() { + testTableSource( + new Json() + .jsonSchema(JSON_SCHEMA) + .failOnMissingField(true) + ); + } + + @Test + public void testTableSourceDerivedSchema() { + testTableSource( + new Json() + .deriveSchema() + .failOnMissingField(true) + ); + } + + private void testTableSource(FormatDescriptor format) { + // construct table source using a builder + + final Map tableJsonMapping = new HashMap<>(); + tableJsonMapping.put("fruit-name", "name"); + tableJsonMapping.put("count", "count"); + tableJsonMapping.put("event-time", "time"); + + final Properties props = new Properties(); + props.put("group.id", "test-group"); + props.put("bootstrap.servers", "localhost:1234"); + + final Map specificOffsets = new HashMap<>(); + specificOffsets.put(new KafkaTopicPartition(TOPIC, 0), 100L); + specificOffsets.put(new KafkaTopicPartition(TOPIC, 1), 123L); + + final KafkaTableSource builderSource = builder() + .forJsonSchema(TableSchema.fromTypeInfo(JsonSchemaConverter.convert(JSON_SCHEMA))) + .failOnMissingField(true) + .withTableToJsonMapping(tableJsonMapping) + .withKafkaProperties(props) + .forTopic(TOPIC) + .fromSpecificOffsets(specificOffsets) + .withSchema( + TableSchema.builder() + .field("fruit-name", Types.STRING) + .field("count", Types.BIG_INT) + .field("event-time", Types.BIG_DEC) + .field("proc-time", Types.SQL_TIMESTAMP) + .build()) + .withProctimeAttribute("proc-time") + .build(); + + // construct table source using descriptors and table source factory + + final Map offsets = new HashMap<>(); + offsets.put(0, 100L); + offsets.put(1, 123L); + + final TestTableSourceDescriptor testDesc = new TestTableSourceDescriptor( + new Kafka() + .version(version()) + .topic(TOPIC) + .properties(props) + .startFromSpecificOffsets(offsets)) + .addFormat(format) + .addSchema( + new Schema() + .field("fruit-name", Types.STRING).from("name") + .field("count", Types.BIG_INT) // no from so it must match with the input + .field("event-time", Types.BIG_DEC).from("time") + .field("proc-time", Types.SQL_TIMESTAMP).proctime()); + + final TableSource factorySource = TableSourceFactoryService.findAndCreateTableSource(testDesc); + + assertEquals(builderSource, factorySource); + } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/table/descriptors/KafkaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/table/descriptors/KafkaTest.java new file mode 100644 index 0000000000000..f3d96f1c443aa --- /dev/null +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/table/descriptors/KafkaTest.java @@ -0,0 +1,113 @@ +/* + * 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.table.descriptors; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * Tests for the {@link Kafka} descriptor. + */ +public class KafkaTest extends DescriptorTestBase { + + @Override + public List descriptors() { + final Descriptor earliestDesc = + new Kafka() + .version("0.8") + .startFromEarliest() + .topic("WhateverTopic"); + + final Descriptor specificOffsetsDesc = + new Kafka() + .version("0.11") + .topic("MyTable") + .startFromSpecificOffset(0, 42L) + .startFromSpecificOffset(1, 300L) + .property("zookeeper.stuff", "12") + .property("kafka.stuff", "42"); + + final Map offsets = new HashMap<>(); + offsets.put(0, 42L); + offsets.put(1, 300L); + + final Properties properties = new Properties(); + properties.put("zookeeper.stuff", "12"); + properties.put("kafka.stuff", "42"); + + final Descriptor specificOffsetsMapDesc = + new Kafka() + .version("0.11") + .topic("MyTable") + .startFromSpecificOffsets(offsets) + .properties(properties); + + return Arrays.asList(earliestDesc, specificOffsetsDesc, specificOffsetsMapDesc); + } + + @Override + public List> properties() { + final Map props1 = new HashMap<>(); + props1.put("connector.property-version", "1"); + props1.put("connector.type", "kafka"); + props1.put("connector.version", "0.8"); + props1.put("connector.topic", "WhateverTopic"); + props1.put("connector.startup-mode", "earliest-offset"); + + final Map props2 = new HashMap<>(); + props2.put("connector.property-version", "1"); + props2.put("connector.type", "kafka"); + props2.put("connector.version", "0.11"); + props2.put("connector.topic", "MyTable"); + props2.put("connector.startup-mode", "specific-offsets"); + props2.put("connector.specific-offsets.0.partition", "0"); + props2.put("connector.specific-offsets.0.offset", "42"); + props2.put("connector.specific-offsets.1.partition", "1"); + props2.put("connector.specific-offsets.1.offset", "300"); + props2.put("connector.properties.0.key", "zookeeper.stuff"); + props2.put("connector.properties.0.value", "12"); + props2.put("connector.properties.1.key", "kafka.stuff"); + props2.put("connector.properties.1.value", "42"); + + final Map props3 = new HashMap<>(); + props3.put("connector.property-version", "1"); + props3.put("connector.type", "kafka"); + props3.put("connector.version", "0.11"); + props3.put("connector.topic", "MyTable"); + props3.put("connector.startup-mode", "specific-offsets"); + props3.put("connector.specific-offsets.0.partition", "0"); + props3.put("connector.specific-offsets.0.offset", "42"); + props3.put("connector.specific-offsets.1.partition", "1"); + props3.put("connector.specific-offsets.1.offset", "300"); + props3.put("connector.properties.0.key", "zookeeper.stuff"); + props3.put("connector.properties.0.value", "12"); + props3.put("connector.properties.1.key", "kafka.stuff"); + props3.put("connector.properties.1.value", "42"); + + return Arrays.asList(props1, props2, props3); + } + + @Override + public DescriptorValidator validator() { + return new KafkaValidator(); + } +} diff --git a/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-json-schema.json b/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-json-schema.json deleted file mode 100644 index 5167e5e872405..0000000000000 --- a/flink-connectors/flink-connector-kafka-base/src/test/resources/kafka-json-schema.json +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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. - */ - -{ - "title": "Fruit", - "type": "object", - "properties": { - "name": { - "type": "string" - }, - "count": { - "type": "integer" - }, - "time": { - "description": "Age in years", - "type": "number" - } - }, - "required": ["name", "count", "time"] -} diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index c555f666bed20..90e79ea155df5 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -337,22 +337,6 @@ under the License. - - - org.apache.flink - flink-avro - ${project.version} - provided - - - - org.apache.flink - flink-json - ${project.version} - provided - - - diff --git a/flink-formats/flink-json/pom.xml b/flink-formats/flink-json/pom.xml index ccc48f4e41bc6..1c77a4535b14f 100644 --- a/flink-formats/flink-json/pom.xml +++ b/flink-formats/flink-json/pom.xml @@ -50,6 +50,33 @@ under the License. ${project.version} provided - + + org.apache.flink + + flink-table_2.11 + ${project.version} + provided + + true + + + + + + org.apache.flink + + flink-table_2.11 + ${project.version} + test-jar + test + + + + + org.scala-lang + scala-compiler + test + + diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java new file mode 100644 index 0000000000000..9c121916d5416 --- /dev/null +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/Json.java @@ -0,0 +1,129 @@ +/* + * 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.table.descriptors; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.typeutils.TypeStringUtils; +import org.apache.flink.util.Preconditions; + +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_DERIVE_SCHEMA; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_FAIL_ON_MISSING_FIELD; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_JSON_SCHEMA; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_SCHEMA; +import static org.apache.flink.table.descriptors.JsonValidator.FORMAT_TYPE_VALUE; + +/** + * Format descriptor for JSON. + */ +public class Json extends FormatDescriptor { + + private Boolean failOnMissingField; + private Boolean deriveSchema; + private String jsonSchema; + private String schema; + + /** + * Format descriptor for JSON. + */ + public Json() { + super(FORMAT_TYPE_VALUE, 1); + } + + /** + * Sets flag whether to fail if a field is missing or not. + * + * @param failOnMissingField If set to true, the operation fails if there is a missing field. + * If set to false, a missing field is set to null. + */ + public Json failOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + return this; + } + + /** + * Sets the JSON schema string with field names and the types according to the JSON schema + * specification [[http://json-schema.org/specification.html]]. + * + *

    The schema might be nested. + * + * @param jsonSchema JSON schema + */ + public Json jsonSchema(String jsonSchema) { + Preconditions.checkNotNull(jsonSchema); + this.jsonSchema = jsonSchema; + this.schema = null; + this.deriveSchema = null; + return this; + } + + /** + * Sets the schema using type information. + * + *

    JSON objects are represented as ROW types. + * + *

    The schema might be nested. + * + * @param schemaType type information that describes the schema + */ + public Json schema(TypeInformation schemaType) { + Preconditions.checkNotNull(schemaType); + this.schema = TypeStringUtils.writeTypeInfo(schemaType); + this.jsonSchema = null; + this.deriveSchema = null; + return this; + } + + /** + * Derives the format schema from the table's schema described using {@link Schema}. + * + *

    This allows for defining schema information only once. + * + *

    The names, types, and field order of the format are determined by the table's + * schema. Time attributes are ignored. A "from" definition is interpreted as a field renaming + * in the format. + */ + public Json deriveSchema() { + this.deriveSchema = true; + this.schema = null; + this.jsonSchema = null; + return this; + } + + /** + * Internal method for format properties conversion. + */ + @Override + public void addFormatProperties(DescriptorProperties properties) { + if (deriveSchema != null) { + properties.putBoolean(FORMAT_DERIVE_SCHEMA(), deriveSchema); + } + + if (jsonSchema != null) { + properties.putString(FORMAT_JSON_SCHEMA, jsonSchema); + } + + if (schema != null) { + properties.putString(FORMAT_SCHEMA, schema); + } + + if (failOnMissingField != null) { + properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, failOnMissingField); + } + } +} diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.java new file mode 100644 index 0000000000000..fea7cf55b5d33 --- /dev/null +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/table/descriptors/JsonValidator.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.table.descriptors; + +import org.apache.flink.table.api.ValidationException; + +/** + * Validator for {@link Json}. + */ +public class JsonValidator extends FormatDescriptorValidator { + + public static final String FORMAT_TYPE_VALUE = "json"; + public static final String FORMAT_SCHEMA = "format.schema"; + public static final String FORMAT_JSON_SCHEMA = "format.json-schema"; + public static final String FORMAT_FAIL_ON_MISSING_FIELD = "format.fail-on-missing-field"; + + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + properties.validateBoolean(FORMAT_DERIVE_SCHEMA(), true); + final boolean deriveSchema = properties.getOptionalBoolean(FORMAT_DERIVE_SCHEMA()).orElse(false); + final boolean hasSchema = properties.containsKey(FORMAT_SCHEMA); + final boolean hasSchemaString = properties.containsKey(FORMAT_JSON_SCHEMA); + if (deriveSchema && (hasSchema || hasSchemaString)) { + throw new ValidationException( + "Format cannot define a schema and derive from the table's schema at the same time."); + } else if (!deriveSchema && hasSchema && hasSchemaString) { + throw new ValidationException("A definition of both a schema and JSON schema is not allowed."); + } else if (!deriveSchema && !hasSchema && !hasSchemaString) { + throw new ValidationException("A definition of a schema or JSON schema is required."); + } else if (hasSchema) { + properties.validateType(FORMAT_SCHEMA, false); + } else if (hasSchemaString) { + properties.validateString(FORMAT_JSON_SCHEMA, false, 1); + } + + properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, true); + } +} diff --git a/flink-formats/flink-json/src/test/java/org/apache/flink/table/descriptors/JsonTest.java b/flink-formats/flink-json/src/test/java/org/apache/flink/table/descriptors/JsonTest.java new file mode 100644 index 0000000000000..6e370a02c13ed --- /dev/null +++ b/flink-formats/flink-json/src/test/java/org/apache/flink/table/descriptors/JsonTest.java @@ -0,0 +1,124 @@ +/* + * 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.table.descriptors; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.Types; +import org.apache.flink.table.api.ValidationException; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Tests for the {@link Json} descriptor. + */ +public class JsonTest extends DescriptorTestBase { + + private static final String JSON_SCHEMA = + "{" + + " 'title': 'Person'," + + " 'type': 'object'," + + " 'properties': {" + + " 'firstName': {" + + " 'type': 'string'" + + " }," + + " 'lastName': {" + + " 'type': 'string'" + + " }," + + " 'age': {" + + " 'description': 'Age in years'," + + " 'type': 'integer'," + + " 'minimum': 0" + + " }" + + " }," + + " 'required': ['firstName', 'lastName']" + + "}"; + + @Test(expected = ValidationException.class) + public void testInvalidMissingField() { + addPropertyAndVerify(descriptors().get(0), "format.fail-on-missing-field", "DDD"); + } + + @Test(expected = ValidationException.class) + public void testMissingSchema() { + removePropertyAndVerify(descriptors().get(0), "format.json-schema"); + } + + @Test(expected = ValidationException.class) + public void testDuplicateSchema() { + // we add an additional non-json schema + addPropertyAndVerify(descriptors().get(0), "format.schema", "DDD"); + } + + // -------------------------------------------------------------------------------------------- + + @Override + public List descriptors() { + final Descriptor desc1 = new Json().jsonSchema("test"); + + final Descriptor desc2 = new Json().jsonSchema(JSON_SCHEMA).failOnMissingField(true); + + final Descriptor desc3 = new Json() + .schema( + Types.ROW( + new String[]{"test1", "test2"}, + new TypeInformation[]{Types.STRING(), Types.SQL_TIMESTAMP()})) + .failOnMissingField(true); + + final Descriptor desc4 = new Json().deriveSchema(); + + return Arrays.asList(desc1, desc2, desc3, desc4); + } + + @Override + public List> properties() { + final Map props1 = new HashMap<>(); + props1.put("format.type", "json"); + props1.put("format.property-version", "1"); + props1.put("format.json-schema", "test"); + + final Map props2 = new HashMap<>(); + props2.put("format.type", "json"); + props2.put("format.property-version", "1"); + props2.put("format.json-schema", JSON_SCHEMA); + props2.put("format.fail-on-missing-field", "true"); + + final Map props3 = new HashMap<>(); + props3.put("format.type", "json"); + props3.put("format.property-version", "1"); + props3.put("format.schema", "ROW(test1 VARCHAR, test2 TIMESTAMP)"); + props3.put("format.fail-on-missing-field", "true"); + + final Map props4 = new HashMap<>(); + props4.put("format.type", "json"); + props4.put("format.property-version", "1"); + props4.put("format.derive-schema", "true"); + + return Arrays.asList(props1, props2, props3, props4); + } + + @Override + public DescriptorValidator validator() { + return new JsonValidator(); + } +} diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index 9e7413c2d3e1a..8c40885d36a72 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -476,7 +476,7 @@ private TableEnvironment createTableEnvironment(Environment env) { } env.getSources().forEach((name, source) -> { - TableSource tableSource = TableSourceFactoryService.findTableSourceFactory(source); + TableSource tableSource = TableSourceFactoryService.findAndCreateTableSource(source); tableEnv.registerTableSource(name, tableSource); }); diff --git a/flink-libraries/flink-table/pom.xml b/flink-libraries/flink-table/pom.xml index bf6585a0be29d..2c7888e633f45 100644 --- a/flink-libraries/flink-table/pom.xml +++ b/flink-libraries/flink-table/pom.xml @@ -317,6 +317,18 @@ under the License. + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + org.scalastyle diff --git a/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties b/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties index 86a48a8667b0e..ec4657988ad62 100644 --- a/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties +++ b/flink-libraries/flink-table/src/main/resources/tableSourceConverter.properties @@ -16,6 +16,13 @@ # limitations under the License. ################################################################################ +################################################################################ +# NOTE: THIS APPROACH IS DEPRECATED AND WILL BE REMOVED IN FUTURE VERSIONS! +# +# We recommend to use a org.apache.flink.table.sources.TableSourceFactory +# instead. They allow to define new factories by using Java Service Providers. +################################################################################ + ################################################################################ # The config file is used to specify the packages of current module where # to find TableSourceConverter implementation class annotated with TableType. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala index 1e88d932ed6f9..6958b3d15da8a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.CompositeType import _root_.scala.collection.mutable.ArrayBuffer +import _root_.java.util.Objects /** * A TableSchema represents a Table's structure. @@ -94,7 +95,7 @@ class TableSchema( /** * Returns the number of columns. */ - def getColumnNum: Int = columnNames.length + def getColumnCount: Int = columnNames.length /** * Returns all column names as an array. @@ -134,6 +135,9 @@ class TableSchema( def canEqual(other: Any): Boolean = other.isInstanceOf[TableSchema] + override def hashCode(): Int = { + Objects.hash(columnNames, columnTypes) + } } object TableSchema { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala index fc7f7a39eab7f..ef14b8af259f2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogTable.scala @@ -23,6 +23,7 @@ import java.util.{HashMap => JHashMap, Map => JMap} import org.apache.flink.table.api.{TableException, TableSchema} import org.apache.flink.table.catalog.ExternalCatalogTable._ +import org.apache.flink.table.descriptors.DescriptorProperties.toScala import org.apache.flink.table.descriptors.MetadataValidator.{METADATA_COMMENT, METADATA_CREATION_TIME, METADATA_LAST_ACCESS_TIME} import org.apache.flink.table.descriptors._ import org.apache.flink.table.plan.stats.TableStats @@ -73,8 +74,7 @@ class ExternalCatalogTable( lazy val tableType: String = { val props = new DescriptorProperties() connectorDesc.addProperties(props) - props - .getString(CONNECTOR_LEGACY_TYPE) + toScala(props.getOptionalString(CONNECTOR_LEGACY_TYPE)) .getOrElse(throw new TableException("Could not find a legacy table type to return.")) } @@ -88,8 +88,7 @@ class ExternalCatalogTable( lazy val schema: TableSchema = { val props = new DescriptorProperties() connectorDesc.addProperties(props) - props - .getTableSchema(CONNECTOR_LEGACY_SCHEMA) + toScala(props.getOptionalTableSchema(CONNECTOR_LEGACY_SCHEMA)) .getOrElse(throw new TableException("Could not find a legacy schema to return.")) } @@ -105,7 +104,7 @@ class ExternalCatalogTable( val props = new DescriptorProperties(normalizeKeys = false) val legacyProps = new JHashMap[String, String]() connectorDesc.addProperties(props) - props.asMap.flatMap { case (k, v) => + props.asMap.asScala.flatMap { case (k, v) => if (k.startsWith(CONNECTOR_LEGACY_PROPERTY)) { // remove "connector.legacy-property-" Some(legacyProps.put(k.substring(CONNECTOR_LEGACY_PROPERTY.length + 1), v)) @@ -138,7 +137,7 @@ class ExternalCatalogTable( metadataDesc match { case Some(meta) => meta.addProperties(normalizedProps) - normalizedProps.getString(METADATA_COMMENT).orNull + normalizedProps.getOptionalString(METADATA_COMMENT).orElse(null) case None => null } @@ -157,7 +156,7 @@ class ExternalCatalogTable( metadataDesc match { case Some(meta) => meta.addProperties(normalizedProps) - normalizedProps.getLong(METADATA_CREATION_TIME).map(v => Long.box(v)).orNull + normalizedProps.getOptionalLong(METADATA_CREATION_TIME).orElse(null) case None => null } @@ -176,7 +175,7 @@ class ExternalCatalogTable( metadataDesc match { case Some(meta) => meta.addProperties(normalizedProps) - normalizedProps.getLong(METADATA_LAST_ACCESS_TIME).map(v => Long.box(v)).orNull + normalizedProps.getOptionalLong(METADATA_LAST_ACCESS_TIME).orElse(null) case None => null } @@ -267,7 +266,7 @@ object ExternalCatalogTable { tableType: String, schema: TableSchema, legacyProperties: JMap[String, String]) - extends ConnectorDescriptor(CONNECTOR_TYPE_VALUE, version = 1) { + extends ConnectorDescriptor(CONNECTOR_TYPE_VALUE, version = 1, formatNeeded = false) { override protected def addConnectorProperties(properties: DescriptorProperties): Unit = { properties.putString(CONNECTOR_LEGACY_TYPE, tableType) @@ -276,8 +275,6 @@ object ExternalCatalogTable { properties.putString(s"$CONNECTOR_LEGACY_PROPERTY-$k", v) } } - - override private[flink] def needsFormat() = false } def toConnectorDescriptor( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala index 3bc5dc067b4d7..2288522b8e489 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalTableSourceUtil.scala @@ -58,7 +58,7 @@ object ExternalTableSourceUtil extends Logging { } // use the factory approach else { - val source = TableSourceFactoryService.findTableSourceFactory(externalCatalogTable) + val source = TableSourceFactoryService.findAndCreateTableSource(externalCatalogTable) tableEnv match { // check for a batch table source in this batch environment case _: BatchTableEnvironment => diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala index ed9ee7dbb9790..afdd84c78a9c2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/BatchTableSourceDescriptor.scala @@ -43,7 +43,7 @@ class BatchTableSourceDescriptor(tableEnv: BatchTableEnvironment, connector: Con * Searches for the specified table source, configures it accordingly, and returns it. */ def toTableSource: TableSource[_] = { - val source = TableSourceFactoryService.findTableSourceFactory(this) + val source = TableSourceFactoryService.findAndCreateTableSource(this) source match { case _: BatchTableSource[_] => source case _ => throw new TableException( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectorDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectorDescriptor.scala index f691b4fd104b5..dc344f31bb594 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectorDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectorDescriptor.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.descriptors -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_VERSION} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_PROPERTY_VERSION} /** * Describes a connector to an other system. @@ -27,7 +27,8 @@ import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTO */ abstract class ConnectorDescriptor( private val tpe: String, - private val version: Int) + private val version: Int, + private val formatNeeded: Boolean) extends Descriptor { override def toString: String = this.getClass.getSimpleName @@ -37,7 +38,7 @@ abstract class ConnectorDescriptor( */ final private[flink] def addProperties(properties: DescriptorProperties): Unit = { properties.putString(CONNECTOR_TYPE, tpe) - properties.putLong(CONNECTOR_VERSION, version) + properties.putLong(CONNECTOR_PROPERTY_VERSION, version) addConnectorProperties(properties) } @@ -49,6 +50,6 @@ abstract class ConnectorDescriptor( /** * Internal method that defines if this connector requires a format descriptor. */ - private[flink] def needsFormat(): Boolean + private[flink] def needsFormat(): Boolean = formatNeeded } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectorDescriptorValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectorDescriptorValidator.scala index 8ab0f45fa6445..211d374de58aa 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectorDescriptorValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ConnectorDescriptorValidator.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.descriptors -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_VERSION} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_PROPERTY_VERSION} /** * Validator for [[ConnectorDescriptor]]. @@ -27,13 +27,27 @@ class ConnectorDescriptorValidator extends DescriptorValidator { override def validate(properties: DescriptorProperties): Unit = { properties.validateString(CONNECTOR_TYPE, isOptional = false, minLen = 1) - properties.validateInt(CONNECTOR_VERSION, isOptional = true, 0, Integer.MAX_VALUE) + properties.validateInt(CONNECTOR_PROPERTY_VERSION, isOptional = true, 0, Integer.MAX_VALUE) } } object ConnectorDescriptorValidator { + /** + * Key for describing the type of the connector. Usually used for factory discovery. + */ val CONNECTOR_TYPE = "connector.type" + + /** + * Key for describing the property version. This property can be used for backwards + * compatibility in case the property format changes. + */ + val CONNECTOR_PROPERTY_VERSION = "connector.property-version" + + /** + * Key for describing the version of the connector. This property can be used for different + * connector versions (e.g. Kafka 0.8 or Kafka 0.11). + */ val CONNECTOR_VERSION = "connector.version" } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Csv.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Csv.scala index 0493d9912a9a3..7e69feabf5433 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Csv.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Csv.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.api.{TableSchema, ValidationException} import org.apache.flink.table.descriptors.CsvValidator._ import scala.collection.mutable +import scala.collection.JavaConverters._ /** * Format descriptor for comma-separated values (CSV). @@ -31,7 +32,7 @@ class Csv extends FormatDescriptor(FORMAT_TYPE_VALUE, version = 1) { private var fieldDelim: Option[String] = None private var lineDelim: Option[String] = None - private val formatSchema: mutable.LinkedHashMap[String, String] = + private val schema: mutable.LinkedHashMap[String, String] = mutable.LinkedHashMap[String, String]() private var quoteCharacter: Option[Character] = None private var commentPrefix: Option[String] = None @@ -67,7 +68,7 @@ class Csv extends FormatDescriptor(FORMAT_TYPE_VALUE, version = 1) { * @param schema the table schema */ def schema(schema: TableSchema): Csv = { - this.formatSchema.clear() + this.schema.clear() DescriptorProperties.normalizeTableSchema(schema).foreach { case (n, t) => field(n, t) } @@ -96,10 +97,10 @@ class Csv extends FormatDescriptor(FORMAT_TYPE_VALUE, version = 1) { * @param fieldType the type string of the field */ def field(fieldName: String, fieldType: String): Csv = { - if (formatSchema.contains(fieldName)) { + if (schema.contains(fieldName)) { throw new ValidationException(s"Duplicate field name $fieldName.") } - formatSchema += (fieldName -> fieldType) + schema += (fieldName -> fieldType) this } @@ -145,7 +146,9 @@ class Csv extends FormatDescriptor(FORMAT_TYPE_VALUE, version = 1) { override protected def addFormatProperties(properties: DescriptorProperties): Unit = { fieldDelim.foreach(properties.putString(FORMAT_FIELD_DELIMITER, _)) lineDelim.foreach(properties.putString(FORMAT_LINE_DELIMITER, _)) - properties.putTableSchema(FORMAT_FIELDS, formatSchema.toIndexedSeq) + properties.putTableSchema( + FORMAT_FIELDS, + schema.toIndexedSeq.map(DescriptorProperties.toJava[String, String]).asJava) quoteCharacter.foreach(properties.putCharacter(FORMAT_QUOTE_CHARACTER, _)) commentPrefix.foreach(properties.putString(FORMAT_COMMENT_PREFIX, _)) isIgnoreFirstLine.foreach(properties.putBoolean(FORMAT_IGNORE_FIRST_LINE, _)) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala index d11273251c943..555d92db6e681 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/DescriptorProperties.scala @@ -21,24 +21,32 @@ package org.apache.flink.table.descriptors import java.io.Serializable import java.lang.{Boolean => JBoolean, Double => JDouble, Integer => JInt, Long => JLong} import java.util +import java.util.function.{Consumer, Supplier} import java.util.regex.Pattern +import java.util.{Optional, List => JList, Map => JMap} import org.apache.commons.codec.binary.Base64 import org.apache.commons.lang.StringEscapeUtils import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.table.api.{TableSchema, ValidationException} -import org.apache.flink.table.descriptors.DescriptorProperties.{NAME, TYPE, normalizeTableSchema} +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} +import org.apache.flink.table.api.{TableException, TableSchema, ValidationException} +import org.apache.flink.table.descriptors.DescriptorProperties.{NAME, TYPE, normalizeTableSchema, toJava} import org.apache.flink.table.typeutils.TypeStringUtils import org.apache.flink.util.InstantiationUtil import org.apache.flink.util.Preconditions.checkNotNull -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable /** * Utility class for having a unified string-based representation of Table API related classes * such as [[TableSchema]], [[TypeInformation]], etc. * + * '''Note to implementers''': Please try to reuse key names as much as possible. Key-names + * should be hierarchical and lower case. Use "-" instead of dots or camel case. + * E.g., connector.schema.start-from = from-earliest. Try not to use the higher level in a + * key-name. E.g., instead of connector.kafka.kafka-version use connector.kafka.version. + * * @param normalizeKeys flag that indicates if keys should be normalized (this flag is * necessary for backwards compatibility) */ @@ -46,39 +54,18 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { private val properties: mutable.Map[String, String] = new mutable.HashMap[String, String]() - private def put(key: String, value: String): Unit = { - if (properties.contains(key)) { - throw new IllegalStateException("Property already present.") - } - if (normalizeKeys) { - properties.put(key.toLowerCase, value) - } else { - properties.put(key, value) - } - } - - // for testing - private[flink] def unsafePut(key: String, value: String): Unit = { - properties.put(key, value) - } - - // for testing - private[flink] def unsafeRemove(key: String): Unit = { - properties.remove(key) - } - - def putProperties(properties: Map[String, String]): Unit = { - properties.foreach { case (k, v) => - put(k, v) - } - } - - def putProperties(properties: util.Map[String, String]): Unit = { + /** + * Adds a set of properties. + */ + def putProperties(properties: JMap[String, String]): Unit = { properties.asScala.foreach { case (k, v) => put(k, v) } } + /** + * Adds a class under the given key. + */ def putClass(key: String, clazz: Class[_]): Unit = { checkNotNull(key) checkNotNull(clazz) @@ -89,43 +76,62 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { put(key, clazz.getName) } + /** + * Adds a string under the given key. + */ def putString(key: String, str: String): Unit = { checkNotNull(key) checkNotNull(str) put(key, str) } + /** + * Adds a boolean under the given key. + */ def putBoolean(key: String, b: Boolean): Unit = { checkNotNull(key) put(key, b.toString) } + /** + * Adds a long under the given key. + */ def putLong(key: String, l: Long): Unit = { checkNotNull(key) put(key, l.toString) } + /** + * Adds an integer under the given key. + */ def putInt(key: String, i: Int): Unit = { checkNotNull(key) put(key, i.toString) } + /** + * Adds a character under the given key. + */ def putCharacter(key: String, c: Character): Unit = { checkNotNull(key) checkNotNull(c) put(key, c.toString) } + /** + * Adds a table schema under the given key. + */ def putTableSchema(key: String, schema: TableSchema): Unit = { + checkNotNull(key) + checkNotNull(schema) putTableSchema(key, normalizeTableSchema(schema)) } - def putTableSchema(key: String, nameAndType: Seq[(String, String)]): Unit = { - putIndexedFixedProperties( - key, - Seq(NAME, TYPE), - nameAndType.map(t => Seq(t._1, t._2)) - ) + /** + * Adds a table schema under the given key. + */ + def putTableSchema(key: String, nameAndType: JList[JTuple2[String, String]]): Unit = { + putTableSchema(key, nameAndType.asScala.map(t => (t.f0, t.f1))) } /** @@ -140,19 +146,12 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { */ def putIndexedFixedProperties( key: String, - propertyKeys: Seq[String], - propertyValues: Seq[Seq[String]]) + propertyKeys: JList[String], + propertyValues: JList[JList[String]]) : Unit = { checkNotNull(key) checkNotNull(propertyValues) - propertyValues.zipWithIndex.foreach { case (values, idx) => - if (values.lengthCompare(propertyKeys.size) != 0) { - throw new ValidationException("Values must have same arity as keys.") - } - values.zipWithIndex.foreach { case (value, keyIdx) => - put(s"$key.$idx.${propertyKeys(keyIdx)}", value) - } - } + putIndexedFixedProperties(key, propertyKeys.asScala, propertyValues.asScala.map(_.asScala)) } /** @@ -167,65 +166,163 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { */ def putIndexedVariableProperties( key: String, - propertySets: Seq[Map[String, String]]) + propertySets: JList[JMap[String, String]]) : Unit = { checkNotNull(key) checkNotNull(propertySets) - propertySets.zipWithIndex.foreach { case (propertySet, idx) => - propertySet.foreach { case (k, v) => - put(s"$key.$idx.$k", v) - } - } + putIndexedVariableProperties(key, propertySets.asScala.map(_.asScala.toMap)) } // ---------------------------------------------------------------------------------------------- - def getString(key: String): Option[String] = { - properties.get(key) + /** + * Returns a string value under the given key if it exists. + */ + def getOptionalString(key: String): Optional[String] = toJava(properties.get(key)) + + /** + * Returns a string value under the given existing key. + */ + def getString(key: String): String = { + get(key) } - def getCharacter(key: String): Option[Character] = getString(key) match { - case Some(c) => + /** + * Returns a character value under the given key if it exists. + */ + def getOptionalCharacter(key: String): Optional[Character] = { + val value = properties.get(key).map { c => if (c.length != 1) { throw new ValidationException(s"The value of $key must only contain one character.") } - Some(c.charAt(0)) + Char.box(c.charAt(0)) + } + toJava(value) + } - case None => None + /** + * Returns a character value under the given existing key. + */ + def getCharacter(key: String): Char = { + getOptionalCharacter(key).orElseThrow(exceptionSupplier(key)) } - def getBoolean(key: String): Option[Boolean] = getString(key) match { - case Some(b) => Some(JBoolean.parseBoolean(b)) + /** + * Returns a class value under the given key if it exists. + */ + def getOptionalClass[T](key: String, superClass: Class[T]): Optional[Class[T]] = { + val value = properties.get(key).map { name => + val clazz = try { + Class.forName( + name, + true, + Thread.currentThread().getContextClassLoader).asInstanceOf[Class[T]] + } catch { + case e: Exception => + throw new ValidationException(s"Could not get class '$name' for key '$key'.", e) + } + if (!superClass.isAssignableFrom(clazz)) { + throw new ValidationException(s"Class '$name' does not extend from the required " + + s"class '${superClass.getName}' for key '$key'.") + } + clazz + } + toJava(value) + } - case None => None + /** + * Returns a class value under the given existing key. + */ + def getClass[T](key: String, superClass: Class[T]): Class[T] = { + getOptionalClass(key, superClass).orElseThrow(exceptionSupplier(key)) } - def getInt(key: String): Option[Int] = getString(key) match { - case Some(l) => Some(JInt.parseInt(l)) + /** + * Returns a boolean value under the given key if it exists. + */ + def getOptionalBoolean(key: String): Optional[JBoolean] = { + val value = properties.get(key).map(JBoolean.parseBoolean(_)).map(Boolean.box) + toJava(value) + } + + /** + * Returns a boolean value under the given existing key. + */ + def getBoolean(key: String): Boolean = { + getOptionalBoolean(key).orElseThrow(exceptionSupplier(key)) + } - case None => None + /** + * Returns an integer value under the given key if it exists. + */ + def getOptionalInt(key: String): Optional[JInt] = { + val value = properties.get(key).map(JInt.parseInt(_)).map(Int.box) + toJava(value) } - def getLong(key: String): Option[Long] = getString(key) match { - case Some(l) => Some(JLong.parseLong(l)) + /** + * Returns an integer value under the given existing key. + */ + def getInt(key: String): Int = { + getOptionalInt(key).orElseThrow(exceptionSupplier(key)) + } - case None => None + /** + * Returns a long value under the given key if it exists. + */ + def getOptionalLong(key: String): Optional[JLong] = { + val value = properties.get(key).map(JLong.parseLong(_)).map(Long.box) + toJava(value) + } + + /** + * Returns a long value under the given existing key. + */ + def getLong(key: String): Long = { + getOptionalLong(key).orElseThrow(exceptionSupplier(key)) + } + + /** + * Returns a double value under the given key if it exists. + */ + def getOptionalDouble(key: String): Optional[JDouble] = { + val value = properties.get(key).map(JDouble.parseDouble(_)).map(Double.box) + toJava(value) + } + + /** + * Returns a double value under the given key if it exists. + */ + def getDouble(key: String): Double = { + getOptionalDouble(key).orElseThrow(exceptionSupplier(key)) } - def getDouble(key: String): Option[Double] = getString(key) match { - case Some(d) => Some(JDouble.parseDouble(d)) + /** + * Returns the type information under the given key if it exists. + */ + def getOptionalType(key: String): Optional[TypeInformation[_]] = { + val value = properties.get(key).map(TypeStringUtils.readTypeInfo) + toJava(value) + } - case None => None + /** + * Returns the type information under the given existing key. + */ + def getType(key: String): TypeInformation[_] = { + getOptionalType(key).orElseThrow(exceptionSupplier(key)) } - def getTableSchema(key: String): Option[TableSchema] = { + /** + * Returns a table schema under the given key if it exists. + */ + def getOptionalTableSchema(key: String): Optional[TableSchema] = { // filter for number of columns val fieldCount = properties .filterKeys(k => k.startsWith(key) && k.endsWith(s".$NAME")) .size if (fieldCount == 0) { - return None + return toJava(None) } // validate fields and build schema @@ -243,16 +340,186 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { ) ) } - Some(schemaBuilder.build()) + toJava(Some(schemaBuilder.build())) + } + + /** + * Returns a table schema under the given existing key. + */ + def getTableSchema(key: String): TableSchema = { + getOptionalTableSchema(key).orElseThrow(exceptionSupplier(key)) + } + + /** + * Returns the property keys of fixed indexed properties. + * + * For example: + * + * schema.fields.0.type = INT, schema.fields.0.name = test + * schema.fields.1.type = LONG, schema.fields.1.name = test2 + * + * getFixedIndexedProperties("schema.fields", List("type", "name")) leads to: + * + * 0: Map("type" -> "schema.fields.0.type", "name" -> "schema.fields.0.name") + * 1: Map("type" -> "schema.fields.1.type", "name" -> "schema.fields.1.name") + */ + def getFixedIndexedProperties( + key: String, + propertyKeys: JList[String]) + : JList[JMap[String, String]] = { + + val keys = propertyKeys.asScala + + // filter for index + val escapedKey = Pattern.quote(key) + val pattern = Pattern.compile(s"$escapedKey\\.(\\d+)\\.(.*)") + + // extract index and property keys + val indexes = properties.keys.flatMap { k => + val matcher = pattern.matcher(k) + if (matcher.find()) { + Some(JInt.parseInt(matcher.group(1))) + } else { + None + } + } + + // determine max index + val maxIndex = indexes.reduceOption(_ max _).getOrElse(-1) + + // validate and create result + val list = new util.ArrayList[JMap[String, String]]() + for (i <- 0 to maxIndex) { + val map = new util.HashMap[String, String]() + + keys.foreach { subKey => + val fullKey = s"$key.$i.$subKey" + // check for existence of full key + if (!containsKey(fullKey)) { + throw exceptionSupplier(fullKey).get() + } + map.put(subKey, fullKey) + } + + list.add(map) + } + list + } + + /** + * Returns the property keys of variable indexed properties. + * + * For example: + * + * schema.fields.0.type = INT, schema.fields.0.name = test + * schema.fields.1.type = LONG + * + * getFixedIndexedProperties("schema.fields", List("type")) leads to: + * + * 0: Map("type" -> "schema.fields.0.type", "name" -> "schema.fields.0.name") + * 1: Map("type" -> "schema.fields.1.type") + */ + def getVariableIndexedProperties( + key: String, + requiredKeys: JList[String]) + : JList[JMap[String, String]] = { + + val keys = requiredKeys.asScala + + // filter for index + val escapedKey = Pattern.quote(key) + val pattern = Pattern.compile(s"$escapedKey\\.(\\d+)\\.(.*)") + + // extract index and property keys + val indexes = properties.keys.flatMap { k => + val matcher = pattern.matcher(k) + if (matcher.find()) { + Some((JInt.parseInt(matcher.group(1)), matcher.group(2))) + } else { + None + } + } + + // determine max index + val maxIndex = indexes.map(_._1).reduceOption(_ max _).getOrElse(-1) + + // validate and create result + val list = new util.ArrayList[JMap[String, String]]() + for (i <- 0 to maxIndex) { + val map = new util.HashMap[String, String]() + + // check and add required keys + keys.foreach { subKey => + val fullKey = s"$key.$i.$subKey" + // check for existence of full key + if (!containsKey(fullKey)) { + throw exceptionSupplier(fullKey).get() + } + map.put(subKey, fullKey) + } + + // add optional keys + indexes.filter(_._1 == i).foreach { case (_, subKey) => + val fullKey = s"$key.$i.$subKey" + map.put(subKey, fullKey) + } + + list.add(map) + } + list + } + + /** + * Returns all properties under a given key that contains an index in between. + * + * E.g. rowtime.0.name -> returns all rowtime.#.name properties + */ + def getIndexedProperty(key: String, property: String): JMap[String, String] = { + val escapedKey = Pattern.quote(key) + properties.filterKeys(k => k.matches(s"$escapedKey\\.\\d+\\.$property")).asJava + } + + /** + * Returns a prefix subset of properties. + */ + def getPrefix(prefixKey: String): JMap[String, String] = { + val prefix = prefixKey + '.' + properties.filterKeys(_.startsWith(prefix)).toSeq.map{ case (k, v) => + k.substring(prefix.length) -> v // remove prefix + }.toMap.asJava } // ---------------------------------------------------------------------------------------------- + /** + * Validates a string property. + */ + def validateString( + key: String, + isOptional: Boolean) + : Unit = { + validateString(key, isOptional, 0, Integer.MAX_VALUE) + } + + /** + * Validates a string property. The boundaries are inclusive. + */ def validateString( key: String, isOptional: Boolean, - minLen: Int = 0, // inclusive - maxLen: Int = Integer.MAX_VALUE) // inclusive + minLen: Int) // inclusive + : Unit = { + validateString(key, isOptional, minLen, Integer.MAX_VALUE) + } + + /** + * Validates a string property. The boundaries are inclusive. + */ + def validateString( + key: String, + isOptional: Boolean, + minLen: Int, // inclusive + maxLen: Int) // inclusive : Unit = { if (!properties.contains(key)) { @@ -269,11 +536,35 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } + /** + * Validates an integer property. + */ + def validateInt( + key: String, + isOptional: Boolean) + : Unit = { + validateInt(key, isOptional, Int.MinValue, Int.MaxValue) + } + + /** + * Validates an integer property. The boundaries are inclusive. + */ def validateInt( key: String, isOptional: Boolean, - min: Int = Int.MinValue, // inclusive - max: Int = Int.MaxValue) // inclusive + min: Int) // inclusive + : Unit = { + validateInt(key, isOptional, min, Int.MaxValue) + } + + /** + * Validates an integer property. The boundaries are inclusive. + */ + def validateInt( + key: String, + isOptional: Boolean, + min: Int, // inclusive + max: Int) // inclusive : Unit = { if (!properties.contains(key)) { @@ -295,11 +586,35 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } + /** + * Validates a long property. + */ + def validateLong( + key: String, + isOptional: Boolean) + : Unit = { + validateLong(key, isOptional, Long.MinValue, Long.MaxValue) + } + + /** + * Validates a long property. The boundaries are inclusive. + */ + def validateLong( + key: String, + isOptional: Boolean, + min: Long) // inclusive + : Unit = { + validateLong(key, isOptional, min, Long.MaxValue) + } + + /** + * Validates a long property. The boundaries are inclusive. + */ def validateLong( key: String, isOptional: Boolean, - min: Long = Long.MinValue, // inclusive - max: Long = Long.MaxValue) // inclusive + min: Long, // inclusive + max: Long) // inclusive : Unit = { if (!properties.contains(key)) { @@ -321,6 +636,9 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } + /** + * Validates that a certain value is present under the given key. + */ def validateValue(key: String, value: String, isOptional: Boolean): Unit = { if (!properties.contains(key)) { if (!isOptional) { @@ -334,6 +652,9 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } + /** + * Validates that a boolean value is present under the given key. + */ def validateBoolean(key: String, isOptional: Boolean): Unit = { if (!properties.contains(key)) { if (!isOptional) { @@ -348,11 +669,35 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } + /** + * Validates a double property. + */ + def validateDouble( + key: String, + isOptional: Boolean) + : Unit = { + validateDouble(key, isOptional, Double.MinValue, Double.MaxValue) + } + + /** + * Validates a double property. The boundaries are inclusive. + */ + def validateDouble( + key: String, + isOptional: Boolean, + min: Double) // inclusive + : Unit = { + validateDouble(key, isOptional, min, Double.MaxValue) + } + + /** + * Validates a double property. The boundaries are inclusive. + */ def validateDouble( key: String, isOptional: Boolean, - min: Double = Double.MinValue, // inclusive - max: Double = Double.MaxValue) // inclusive + min: Double, // inclusive + max: Double) // inclusive : Unit = { if (!properties.contains(key)) { @@ -374,25 +719,117 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } + /** + * Validation for variable indexed properties. + * + * For example: + * + * schema.fields.0.type = INT, schema.fields.0.name = test + * schema.fields.1.type = LONG + * + * The propertyKeys map defines e.g. "type" and a validation logic for the given full key. + * + * The validation consumer takes the current prefix e.g. "schema.fields.1.". + */ + def validateVariableIndexedProperties( + key: String, + allowEmpty: Boolean, + propertyKeys: JMap[String, Consumer[String]], + requiredKeys: JList[String]) + : Unit = { + + val keys = propertyKeys.asScala + + // filter for index + val escapedKey = Pattern.quote(key) + val pattern = Pattern.compile(s"$escapedKey\\.(\\d+)\\.(.*)") + + // extract index and property keys + val indexes = properties.keys.flatMap { k => + val matcher = pattern.matcher(k) + if (matcher.find()) { + Some(JInt.parseInt(matcher.group(1))) + } else { + None + } + } + + // determine max index + val maxIndex = indexes.reduceOption(_ max _).getOrElse(-1) + + if (maxIndex < 0 && !allowEmpty) { + throw new ValidationException(s"Property key '$key' must not be empty.") + } + + // validate + for (i <- 0 to maxIndex) { + keys.foreach { case (subKey, validation) => + val fullKey = s"$key.$i.$subKey" + // only validate if it exists + if (properties.contains(fullKey)) { + validation.accept(s"$key.$i.") + } else { + // check if it is required + if (requiredKeys.contains(subKey)) { + throw new ValidationException(s"Required property key '$fullKey' is missing.") + } + } + } + } + } + + /** + * Validation for fixed indexed properties. + * + * For example: + * + * schema.fields.0.type = INT, schema.fields.0.name = test + * schema.fields.1.type = LONG, schema.fields.1.name = test2 + * + * The propertyKeys map must define e.g. "type" and "name" and a validation logic for the + * given full key. + */ + def validateFixedIndexedProperties( + key: String, + allowEmpty: Boolean, + propertyKeys: JMap[String, Consumer[String]]) + : Unit = { + + validateVariableIndexedProperties( + key, + allowEmpty, + propertyKeys, + new util.ArrayList(propertyKeys.keySet())) + } + + /** + * Validates a table schema property. + */ def validateTableSchema(key: String, isOptional: Boolean): Unit = { - // filter for name columns - val names = getIndexedProperty(key, NAME) - // filter for type columns - val types = getIndexedProperty(key, TYPE) - if (names.isEmpty && types.isEmpty && !isOptional) { - throw new ValidationException( - s"Could not find the required schema for property '$key'.") + val nameValidation = (prefix: String) => { + validateString(prefix + NAME, isOptional = false, minLen = 1) } - for (i <- 0 until Math.max(names.size, types.size)) { - validateString(s"$key.$i.$NAME", isOptional = false, minLen = 1) - validateType(s"$key.$i.$TYPE", isOptional = false) + val typeValidation = (prefix: String) => { + validateType(prefix + TYPE, isOptional = false) } + + validateFixedIndexedProperties( + key, + isOptional, + Map( + NAME -> toJava(nameValidation), + TYPE -> toJava(typeValidation) + ).asJava + ) } + /** + * Validates a enum property with a set of validation logic for each enum value. + */ def validateEnum( key: String, isOptional: Boolean, - enumToValidation: Map[String, () => Unit]) + enumToValidation: JMap[String, Consumer[String]]) : Unit = { if (!properties.contains(key)) { @@ -401,15 +838,26 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } else { val value = properties(key) - if (!enumToValidation.contains(value)) { + if (!enumToValidation.containsKey(value)) { throw new ValidationException(s"Unknown value for property '$key'. " + - s"Supported values [${enumToValidation.keys.mkString(", ")}] but was: $value") + s"Supported values [${enumToValidation.keySet().asScala.mkString(", ")}] but was: $value") } else { - enumToValidation(value).apply() // run validation logic + // run validation logic + enumToValidation.get(value).accept(key) } } } + /** + * Validates a enum property with a set of enum values. + */ + def validateEnumValues(key: String, isOptional: Boolean, values: JList[String]): Unit = { + validateEnum(key, isOptional, values.asScala.map((_, noValidation())).toMap.asJava) + } + + /** + * Validates a type property. + */ def validateType(key: String, isOptional: Boolean): Unit = { if (!properties.contains(key)) { if (!isOptional) { @@ -420,6 +868,9 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } + /** + * Validates that the given prefix is not included in these properties. + */ def validatePrefixExclusion(prefix: String): Unit = { val invalidField = properties.find(_._1.startsWith(prefix)) if (invalidField.isDefined) { @@ -428,6 +879,9 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { } } + /** + * Validates that the given key is not included in these properties. + */ def validateExclusion(key: String): Unit = { if (properties.contains(key)) { throw new ValidationException(s"Property '$key' is not allowed in this context.") @@ -436,28 +890,159 @@ class DescriptorProperties(normalizeKeys: Boolean = true) { // ---------------------------------------------------------------------------------------------- - def getIndexedProperty(key: String, property: String): Map[String, String] = { - val escapedKey = Pattern.quote(key) - properties.filterKeys(k => k.matches(s"$escapedKey\\.\\d+\\.$property")).toMap + /** + * Returns if any property contains parts of a given string. + */ + def containsString(str: String): Boolean = { + properties.exists(e => e._1.contains(str)) } - def contains(str: String): Boolean = { - properties.exists(e => e._1.contains(str)) + /** + * Returns if the given key is contained. + */ + def containsKey(key: String): Boolean = { + properties.contains(key) } + /** + * Returns if a given prefix exists in the properties. + */ def hasPrefix(prefix: String): Boolean = { properties.exists(e => e._1.startsWith(prefix)) } - def asMap: Map[String, String] = { - properties.toMap + /** + * Returns a Scala Map. + */ + def asMap: JMap[String, String] = { + properties.toMap.asJava + } + + // ---------------------------------------------------------------------------------------------- + + /** + * Returns an empty validation logic. + */ + def noValidation(): Consumer[String] = DescriptorProperties.emptyConsumer + + def exceptionSupplier(key: String): Supplier[TableException] = new Supplier[TableException] { + override def get(): TableException = { + new TableException(s"Property with key '$key' could not be found. " + + s"This is a bug because the validation logic should have checked that before.") + } + } + + // ---------------------------------------------------------------------------------------------- + + /** + * Adds a property. + */ + private def put(key: String, value: String): Unit = { + if (properties.contains(key)) { + throw new IllegalStateException("Property already present.") + } + if (normalizeKeys) { + properties.put(key.toLowerCase, value) + } else { + properties.put(key, value) + } + } + + /** + * Gets an existing property. + */ + private def get(key: String): String = { + properties.getOrElse( + key, + throw exceptionSupplier(key).get()) + } + + /** + * Raw access to the underlying properties map for testing purposes. + */ + private[flink] def unsafePut(key: String, value: String): Unit = { + properties.put(key, value) + } + + /** + * Raw access to the underlying properties map for testing purposes. + */ + private[flink] def unsafeRemove(key: String): Unit = { + properties.remove(key) + } + + /** + * Adds a table schema under the given key. + */ + private def putTableSchema(key: String, nameAndType: Seq[(String, String)]): Unit = { + putIndexedFixedProperties( + key, + Seq(NAME, TYPE), + nameAndType.map(t => Seq(t._1, t._2)) + ) + } + + /** + * Adds an indexed sequence of properties (with sub-properties) under a common key. + * + * For example: + * + * schema.fields.0.type = INT, schema.fields.0.name = test + * schema.fields.1.type = LONG, schema.fields.1.name = test2 + * + * The arity of each propertyValue must match the arity of propertyKeys. + */ + private def putIndexedFixedProperties( + key: String, + propertyKeys: Seq[String], + propertyValues: Seq[Seq[String]]) + : Unit = { + checkNotNull(key) + checkNotNull(propertyValues) + propertyValues.zipWithIndex.foreach { case (values, idx) => + if (values.lengthCompare(propertyKeys.size) != 0) { + throw new ValidationException("Values must have same arity as keys.") + } + values.zipWithIndex.foreach { case (value, keyIdx) => + put(s"$key.$idx.${propertyKeys(keyIdx)}", value) + } + } + } + + /** + * Adds an indexed mapping of properties under a common key. + * + * For example: + * + * schema.fields.0.type = INT, schema.fields.0.name = test + * schema.fields.1.name = test2 + * + * The arity of the propertySets can differ. + */ + private def putIndexedVariableProperties( + key: String, + propertySets: Seq[Map[String, String]]) + : Unit = { + checkNotNull(key) + checkNotNull(propertySets) + propertySets.zipWithIndex.foreach { case (propertySet, idx) => + propertySet.foreach { case (k, v) => + put(s"$key.$idx.$k", v) + } + } } } object DescriptorProperties { - val TYPE = "type" - val NAME = "name" + private val emptyConsumer: Consumer[String] = new Consumer[String] { + override def accept(t: String): Unit = { + // nothing to do + } + } + + val TYPE: String = "type" + val NAME: String = "name" // the string representation should be equal to SqlTypeName def normalizeTypeInfo(typeInfo: TypeInformation[_]): String = { @@ -487,6 +1072,24 @@ object DescriptorProperties { } } + def deserialize[T](data: String, expected: Class[T]): T = { + try { + val byteData = Base64.decodeBase64(data) + val obj = InstantiationUtil.deserializeObject[T]( + byteData, + Thread.currentThread.getContextClassLoader) + if (!expected.isAssignableFrom(obj.getClass)) { + throw new ValidationException( + s"Serialized data contains an object of unexpected type. " + + s"Expected '${expected.getName}' but was '${obj.getClass.getName}'") + } + obj + } catch { + case e: Exception => + throw new ValidationException(s"Could not deserialize data: '$data'", e) + } + } + def toString(keyOrValue: String): String = { StringEscapeUtils.escapeJava(keyOrValue) } @@ -494,4 +1097,24 @@ object DescriptorProperties { def toString(key: String, value: String): String = { toString(key) + "=" + toString(value) } + + // the following methods help for Scala <-> Java interfaces + // most of these methods are not necessary once we upgraded to Scala 2.12 + + def toJava[T](option: Option[T]): Optional[T] = option match { + case Some(v) => Optional.of(v) + case None => Optional.empty() + } + + def toScala[T](option: Optional[T]): Option[T] = Option(option.orElse(null.asInstanceOf[T])) + + def toJava[T](func: Function[T, Unit]): Consumer[T] = new Consumer[T] { + override def accept(t: T): Unit = { + func.apply(t) + } + } + + def toJava[T0, T1](tuple: (T0, T1)): JTuple2[T0, T1] = { + new JTuple2[T0, T1](tuple._1, tuple._2) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala index b1d900f8e316f..f306b5aa72a50 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FileSystem.scala @@ -23,7 +23,8 @@ import org.apache.flink.table.descriptors.FileSystemValidator.{CONNECTOR_PATH, C /** * Connector descriptor for a file system. */ -class FileSystem extends ConnectorDescriptor(CONNECTOR_TYPE_VALUE, version = 1) { +class FileSystem extends ConnectorDescriptor( + CONNECTOR_TYPE_VALUE, version = 1, formatNeeded = true) { private var path: Option[String] = None @@ -43,8 +44,6 @@ class FileSystem extends ConnectorDescriptor(CONNECTOR_TYPE_VALUE, version = 1) override protected def addConnectorProperties(properties: DescriptorProperties): Unit = { path.foreach(properties.putString(CONNECTOR_PATH, _)) } - - override private[flink] def needsFormat() = true } /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptor.scala index 86f6229f903cd..bca67c6ea5799 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptor.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.descriptors -import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_TYPE, FORMAT_VERSION} +import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_TYPE, FORMAT_PROPERTY_VERSION} /** * Describes the format of data. @@ -37,7 +37,7 @@ abstract class FormatDescriptor( */ final private[flink] def addProperties(properties: DescriptorProperties): Unit = { properties.putString(FORMAT_TYPE, tpe) - properties.putInt(FORMAT_VERSION, version) + properties.putInt(FORMAT_PROPERTY_VERSION, version) addFormatProperties(properties) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptorValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptorValidator.scala index 1aaa39987edeb..301189a17966b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptorValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/FormatDescriptorValidator.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.descriptors -import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_TYPE, FORMAT_VERSION} +import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_PROPERTY_VERSION, FORMAT_TYPE} /** * Validator for [[FormatDescriptor]]. @@ -27,13 +27,32 @@ class FormatDescriptorValidator extends DescriptorValidator { override def validate(properties: DescriptorProperties): Unit = { properties.validateString(FORMAT_TYPE, isOptional = false, minLen = 1) - properties.validateInt(FORMAT_VERSION, isOptional = true, 0, Integer.MAX_VALUE) + properties.validateInt(FORMAT_PROPERTY_VERSION, isOptional = true, 0, Integer.MAX_VALUE) } } object FormatDescriptorValidator { + /** + * Key for describing the type of the format. Usually used for factory discovery. + */ val FORMAT_TYPE = "format.type" + + /** + * Key for describing the property version. This property can be used for backwards + * compatibility in case the property format changes. + */ + val FORMAT_PROPERTY_VERSION = "format.property-version" + + /** + * Key for describing the version of the format. This property can be used for different + * format versions (e.g. Avro 1.8.2 or Avro 2.0). + */ val FORMAT_VERSION = "format.version" + /** + * Key for deriving the schema of the format from the table's schema. + */ + val FORMAT_DERIVE_SCHEMA = "format.derive-schema" + } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Json.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Json.scala deleted file mode 100644 index cc46d9cfc601b..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Json.scala +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.table.descriptors - -import org.apache.flink.table.descriptors.JsonValidator.{FORMAT_FAIL_ON_MISSING_FIELD, FORMAT_SCHEMA_STRING, FORMAT_TYPE_VALUE} - -/** - * Encoding descriptor for JSON. - */ -class Json extends FormatDescriptor(FORMAT_TYPE_VALUE, version = 1) { - - private var failOnMissingField: Option[Boolean] = None - - private var schema: Option[String] = None - - /** - * Sets flag whether to fail if a field is missing or not. - * - * @param failOnMissingField If set to true, the operation fails if there is a missing field. - * If set to false, a missing field is set to null. - * @return The builder. - */ - def failOnMissingField(failOnMissingField: Boolean): Json = { - this.failOnMissingField = Some(failOnMissingField) - this - } - - /** - * Sets the JSON schema string with field names and the types according to the JSON schema - * specification [[http://json-schema.org/specification.html]]. Required. - * - * The schema might be nested. - * - * @param schema JSON schema - */ - def schema(schema: String): Json = { - this.schema = Some(schema) - this - } - - /** - * Internal method for format properties conversion. - */ - override protected def addFormatProperties(properties: DescriptorProperties): Unit = { - // we distinguish between "schema string" and "schema" to allow parsing of a - // schema object in the future (such that the entire JSON schema can be defined in a YAML - // file instead of one large string) - schema.foreach(properties.putString(FORMAT_SCHEMA_STRING, _)) - failOnMissingField.foreach(properties.putBoolean(FORMAT_FAIL_ON_MISSING_FIELD, _)) - } -} - -/** - * Encoding descriptor for JSON. - */ -object Json { - - /** - * Encoding descriptor for JSON. - */ - def apply(): Json = new Json() -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/JsonValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/JsonValidator.scala deleted file mode 100644 index 9f11caf128f8f..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/JsonValidator.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.table.descriptors - -import org.apache.flink.table.descriptors.JsonValidator.{FORMAT_FAIL_ON_MISSING_FIELD, FORMAT_SCHEMA_STRING} - -/** - * Validator for [[Json]]. - */ -class JsonValidator extends FormatDescriptorValidator { - - override def validate(properties: DescriptorProperties): Unit = { - super.validate(properties) - properties.validateString(FORMAT_SCHEMA_STRING, isOptional = false, minLen = 1) - properties.validateBoolean(FORMAT_FAIL_ON_MISSING_FIELD, isOptional = true) - } -} - -object JsonValidator { - - val FORMAT_TYPE_VALUE = "json" - val FORMAT_SCHEMA_STRING = "format.schema-string" - val FORMAT_FAIL_ON_MISSING_FIELD = "format.fail-on-missing-field" - -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/MetadataValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/MetadataValidator.scala index a8d580c5d316d..6631e22e8d0de 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/MetadataValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/MetadataValidator.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.descriptors -import org.apache.flink.table.descriptors.MetadataValidator.{METADATA_COMMENT, METADATA_CREATION_TIME, METADATA_LAST_ACCESS_TIME, METADATA_VERSION} +import org.apache.flink.table.descriptors.MetadataValidator.{METADATA_COMMENT, METADATA_CREATION_TIME, METADATA_LAST_ACCESS_TIME, METADATA_PROPERTY_VERSION} /** * Validator for [[Metadata]]. @@ -26,7 +26,7 @@ import org.apache.flink.table.descriptors.MetadataValidator.{METADATA_COMMENT, M class MetadataValidator extends DescriptorValidator { override def validate(properties: DescriptorProperties): Unit = { - properties.validateInt(METADATA_VERSION, isOptional = true, 0, Integer.MAX_VALUE) + properties.validateInt(METADATA_PROPERTY_VERSION, isOptional = true, 0, Integer.MAX_VALUE) properties.validateString(METADATA_COMMENT, isOptional = true) properties.validateLong(METADATA_CREATION_TIME, isOptional = true) properties.validateLong(METADATA_LAST_ACCESS_TIME, isOptional = true) @@ -35,7 +35,7 @@ class MetadataValidator extends DescriptorValidator { object MetadataValidator { - val METADATA_VERSION = "metadata.version" + val METADATA_PROPERTY_VERSION = "metadata.property-version" val METADATA_COMMENT = "metadata.comment" val METADATA_CREATION_TIME = "metadata.creation-time" val METADATA_LAST_ACCESS_TIME = "metadata.last-access-time" diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Rowtime.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Rowtime.scala index a1c80f5840996..ed3854df36b64 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Rowtime.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Rowtime.scala @@ -19,11 +19,12 @@ package org.apache.flink.table.descriptors import org.apache.flink.table.api.Types -import org.apache.flink.table.descriptors.RowtimeValidator.{ROWTIME, ROWTIME_VERSION, normalizeTimestampExtractor, normalizeWatermarkStrategy} +import org.apache.flink.table.descriptors.RowtimeValidator.{normalizeTimestampExtractor, normalizeWatermarkStrategy} import org.apache.flink.table.sources.tsextractors.{ExistingField, StreamRecordTimestamp, TimestampExtractor} import org.apache.flink.table.sources.wmstrategies.{AscendingTimestamps, BoundedOutOfOrderTimestamps, PreserveWatermarks, WatermarkStrategy} import scala.collection.mutable +import scala.collection.JavaConverters._ /** * Rowtime descriptor for describing an event time attribute in the schema. @@ -111,12 +112,9 @@ class Rowtime extends Descriptor { */ final override def addProperties(properties: DescriptorProperties): Unit = { val props = mutable.HashMap[String, String]() - props.put(ROWTIME_VERSION, "1") timestampExtractor.foreach(normalizeTimestampExtractor(_).foreach(e => props.put(e._1, e._2))) watermarkStrategy.foreach(normalizeWatermarkStrategy(_).foreach(e => props.put(e._1, e._2))) - - // use a list for the rowtime to support multiple rowtime attributes in the future - properties.putIndexedVariableProperties(ROWTIME, Seq(props.toMap)) + properties.putProperties(props.toMap.asJava) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala index 74e49f14d84c4..fdec82008aee9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala @@ -18,58 +18,62 @@ package org.apache.flink.table.descriptors -import org.apache.flink.table.descriptors.DescriptorProperties.serialize +import org.apache.flink.table.descriptors.DescriptorProperties.{serialize, toJava} import org.apache.flink.table.descriptors.RowtimeValidator._ import org.apache.flink.table.sources.tsextractors.{ExistingField, StreamRecordTimestamp, TimestampExtractor} import org.apache.flink.table.sources.wmstrategies.{AscendingTimestamps, BoundedOutOfOrderTimestamps, PreserveWatermarks, WatermarkStrategy} +import scala.collection.JavaConverters._ + /** * Validator for [[Rowtime]]. */ class RowtimeValidator(val prefix: String = "") extends DescriptorValidator { override def validate(properties: DescriptorProperties): Unit = { - properties.validateInt(prefix + ROWTIME_VERSION, isOptional = true, 0, Integer.MAX_VALUE) - - val noValidation = () => {} - - val timestampExistingField = () => { - properties.validateString(prefix + TIMESTAMPS_FROM, isOptional = false, minLen = 1) + val timestampExistingField = (_: String) => { + properties.validateString( + prefix + ROWTIME_TIMESTAMPS_FROM, isOptional = false, minLen = 1) } - val timestampCustom = () => { - properties.validateString(prefix + TIMESTAMPS_CLASS, isOptional = false, minLen = 1) - properties.validateString(prefix + TIMESTAMPS_SERIALIZED, isOptional = false, minLen = 1) + val timestampCustom = (_: String) => { + properties.validateString( + prefix + ROWTIME_TIMESTAMPS_CLASS, isOptional = false, minLen = 1) + properties.validateString( + prefix + ROWTIME_TIMESTAMPS_SERIALIZED, isOptional = false, minLen = 1) } properties.validateEnum( - prefix + TIMESTAMPS_TYPE, + prefix + ROWTIME_TIMESTAMPS_TYPE, isOptional = false, Map( - TIMESTAMPS_TYPE_VALUE_FROM_FIELD -> timestampExistingField, - TIMESTAMPS_TYPE_VALUE_FROM_SOURCE -> noValidation, - TIMESTAMPS_TYPE_VALUE_CUSTOM -> timestampCustom - ) + ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD -> toJava(timestampExistingField), + ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_SOURCE -> properties.noValidation(), + ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM -> toJava(timestampCustom) + ).asJava ) - val watermarkPeriodicBounding = () => { - properties.validateLong(prefix + WATERMARKS_DELAY, isOptional = false, min = 0) + val watermarkPeriodicBounded = (_: String) => { + properties.validateLong( + prefix + ROWTIME_WATERMARKS_DELAY, isOptional = false, min = 0) } - val watermarkCustom = () => { - properties.validateString(prefix + WATERMARKS_CLASS, isOptional = false, minLen = 1) - properties.validateString(prefix + WATERMARKS_SERIALIZED, isOptional = false, minLen = 1) + val watermarkCustom = (_: String) => { + properties.validateString( + prefix + ROWTIME_WATERMARKS_CLASS, isOptional = false, minLen = 1) + properties.validateString( + prefix + ROWTIME_WATERMARKS_SERIALIZED, isOptional = false, minLen = 1) } properties.validateEnum( - prefix + WATERMARKS_TYPE, + prefix + ROWTIME_WATERMARKS_TYPE, isOptional = false, Map( - WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING -> noValidation, - WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDING -> watermarkPeriodicBounding, - WATERMARKS_TYPE_VALUE_FROM_SOURCE -> noValidation, - WATERMARKS_TYPE_VALUE_CUSTOM -> watermarkCustom - ) + ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING -> properties.noValidation(), + ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDED -> toJava(watermarkPeriodicBounded), + ROWTIME_WATERMARKS_TYPE_VALUE_FROM_SOURCE -> properties.noValidation(), + ROWTIME_WATERMARKS_TYPE_VALUE_CUSTOM -> toJava(watermarkCustom) + ).asJava ) } } @@ -77,58 +81,113 @@ class RowtimeValidator(val prefix: String = "") extends DescriptorValidator { object RowtimeValidator { val ROWTIME = "rowtime" - - // per rowtime properties - - val ROWTIME_VERSION = "version" - val TIMESTAMPS_TYPE = "timestamps.type" - val TIMESTAMPS_TYPE_VALUE_FROM_FIELD = "from-field" - val TIMESTAMPS_TYPE_VALUE_FROM_SOURCE = "from-source" - val TIMESTAMPS_TYPE_VALUE_CUSTOM = "custom" - val TIMESTAMPS_FROM = "timestamps.from" - val TIMESTAMPS_CLASS = "timestamps.class" - val TIMESTAMPS_SERIALIZED = "timestamps.serialized" - - val WATERMARKS_TYPE = "watermarks.type" - val WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING = "periodic-ascending" - val WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDING = "periodic-bounding" - val WATERMARKS_TYPE_VALUE_FROM_SOURCE = "from-source" - val WATERMARKS_TYPE_VALUE_CUSTOM = "custom" - val WATERMARKS_CLASS = "watermarks.class" - val WATERMARKS_SERIALIZED = "watermarks.serialized" - val WATERMARKS_DELAY = "watermarks.delay" + val ROWTIME_TIMESTAMPS_TYPE = "rowtime.timestamps.type" + val ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD = "from-field" + val ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_SOURCE = "from-source" + val ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM = "custom" + val ROWTIME_TIMESTAMPS_FROM = "rowtime.timestamps.from" + val ROWTIME_TIMESTAMPS_CLASS = "rowtime.timestamps.class" + val ROWTIME_TIMESTAMPS_SERIALIZED = "rowtime.timestamps.serialized" + + val ROWTIME_WATERMARKS_TYPE = "rowtime.watermarks.type" + val ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING = "periodic-ascending" + val ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDED = "periodic-bounded" + val ROWTIME_WATERMARKS_TYPE_VALUE_FROM_SOURCE = "from-source" + val ROWTIME_WATERMARKS_TYPE_VALUE_CUSTOM = "custom" + val ROWTIME_WATERMARKS_CLASS = "rowtime.watermarks.class" + val ROWTIME_WATERMARKS_SERIALIZED = "rowtime.watermarks.serialized" + val ROWTIME_WATERMARKS_DELAY = "rowtime.watermarks.delay" // utilities def normalizeTimestampExtractor(extractor: TimestampExtractor): Map[String, String] = extractor match { + case existing: ExistingField => Map( - TIMESTAMPS_TYPE -> TIMESTAMPS_TYPE_VALUE_FROM_FIELD, - TIMESTAMPS_FROM -> existing.getArgumentFields.apply(0)) + ROWTIME_TIMESTAMPS_TYPE -> ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD, + ROWTIME_TIMESTAMPS_FROM -> existing.getArgumentFields.apply(0)) + case _: StreamRecordTimestamp => - Map(TIMESTAMPS_TYPE -> TIMESTAMPS_TYPE_VALUE_FROM_SOURCE) + Map(ROWTIME_TIMESTAMPS_TYPE -> ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_SOURCE) + case _: TimestampExtractor => Map( - TIMESTAMPS_TYPE -> TIMESTAMPS_TYPE_VALUE_CUSTOM, - TIMESTAMPS_CLASS -> extractor.getClass.getName, - TIMESTAMPS_SERIALIZED -> serialize(extractor)) + ROWTIME_TIMESTAMPS_TYPE -> ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM, + ROWTIME_TIMESTAMPS_CLASS -> extractor.getClass.getName, + ROWTIME_TIMESTAMPS_SERIALIZED -> serialize(extractor)) } def normalizeWatermarkStrategy(strategy: WatermarkStrategy): Map[String, String] = strategy match { + case _: AscendingTimestamps => - Map(WATERMARKS_TYPE -> WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING) - case bounding: BoundedOutOfOrderTimestamps => + Map(ROWTIME_WATERMARKS_TYPE -> ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING) + + case bounded: BoundedOutOfOrderTimestamps => Map( - WATERMARKS_TYPE -> WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDING, - WATERMARKS_DELAY -> bounding.delay.toString) + ROWTIME_WATERMARKS_TYPE -> ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDED, + ROWTIME_WATERMARKS_DELAY -> bounded.delay.toString) + case _: PreserveWatermarks => - Map(WATERMARKS_TYPE -> WATERMARKS_TYPE_VALUE_FROM_SOURCE) + Map(ROWTIME_WATERMARKS_TYPE -> ROWTIME_WATERMARKS_TYPE_VALUE_FROM_SOURCE) + case _: WatermarkStrategy => Map( - WATERMARKS_TYPE -> WATERMARKS_TYPE_VALUE_CUSTOM, - WATERMARKS_CLASS -> strategy.getClass.getName, - WATERMARKS_SERIALIZED -> serialize(strategy)) + ROWTIME_WATERMARKS_TYPE -> ROWTIME_WATERMARKS_TYPE_VALUE_CUSTOM, + ROWTIME_WATERMARKS_CLASS -> strategy.getClass.getName, + ROWTIME_WATERMARKS_SERIALIZED -> serialize(strategy)) } + + def getRowtimeComponents(properties: DescriptorProperties, prefix: String) + : Option[(TimestampExtractor, WatermarkStrategy)] = { + + // create timestamp extractor + val t = properties.getOptionalString(prefix + ROWTIME_TIMESTAMPS_TYPE) + if (!t.isPresent) { + return None + } + val extractor: TimestampExtractor = t.get() match { + + case ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD => + val field = properties.getString(prefix + ROWTIME_TIMESTAMPS_FROM) + new ExistingField(field) + + case ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_SOURCE => + new StreamRecordTimestamp + + case ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM => + val clazz = properties.getClass( + ROWTIME_TIMESTAMPS_CLASS, + classOf[TimestampExtractor]) + DescriptorProperties.deserialize( + properties.getString(prefix + ROWTIME_TIMESTAMPS_SERIALIZED), + clazz) + } + + // create watermark strategy + val s = properties.getString(prefix + ROWTIME_WATERMARKS_TYPE) + val strategy: WatermarkStrategy = s match { + + case ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_ASCENDING => + new AscendingTimestamps() + + case ROWTIME_WATERMARKS_TYPE_VALUE_PERIODIC_BOUNDED => + val delay = properties.getLong(prefix + ROWTIME_WATERMARKS_DELAY) + new BoundedOutOfOrderTimestamps(delay) + + case ROWTIME_WATERMARKS_TYPE_VALUE_FROM_SOURCE => + PreserveWatermarks.INSTANCE + + case ROWTIME_WATERMARKS_TYPE_VALUE_CUSTOM => + val clazz = properties.getClass( + prefix + ROWTIME_WATERMARKS_CLASS, + classOf[WatermarkStrategy]) + DescriptorProperties.deserialize( + properties.getString(prefix + ROWTIME_WATERMARKS_SERIALIZED), + clazz) + } + + Some((extractor, strategy)) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Schema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Schema.scala index 2f3a3897a4af1..fcbb2c7ea2291 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Schema.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Schema.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.descriptors.DescriptorProperties.{normalizeTableSc import org.apache.flink.table.descriptors.SchemaValidator._ import scala.collection.mutable +import scala.collection.JavaConverters._ /** * Describes a schema of a table. @@ -80,7 +81,7 @@ class Schema extends Descriptor { } val fieldProperties = mutable.LinkedHashMap[String, String]() - fieldProperties += (TYPE -> fieldType) + fieldProperties += (SCHEMA_TYPE -> fieldType) tableSchema += (fieldName -> fieldProperties) @@ -100,7 +101,7 @@ class Schema extends Descriptor { lastField match { case None => throw new ValidationException("No field previously defined. Use field() before.") case Some(f) => - tableSchema(f) += (FROM -> originFieldName) + tableSchema(f) += (SCHEMA_FROM -> originFieldName) lastField = None } this @@ -115,7 +116,7 @@ class Schema extends Descriptor { lastField match { case None => throw new ValidationException("No field defined previously. Use field() before.") case Some(f) => - tableSchema(f) += (PROCTIME -> PROCTIME_VALUE_TRUE) + tableSchema(f) += (SCHEMA_PROCTIME -> "true") lastField = None } this @@ -132,7 +133,7 @@ class Schema extends Descriptor { case Some(f) => val fieldProperties = new DescriptorProperties() rowtime.addProperties(fieldProperties) - tableSchema(f) ++= fieldProperties.asMap + tableSchema(f) ++= fieldProperties.asMap.asScala lastField = None } this @@ -142,12 +143,11 @@ class Schema extends Descriptor { * Internal method for properties conversion. */ final override private[flink] def addProperties(properties: DescriptorProperties): Unit = { - properties.putInt(SCHEMA_VERSION, 1) properties.putIndexedVariableProperties( SCHEMA, tableSchema.toSeq.map { case (name, props) => - Map(NAME -> name) ++ props - } + (Map(SCHEMA_NAME -> name) ++ props).asJava + }.asJava ) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala index 19c0e411dd233..0a2391175bf01 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala @@ -18,9 +18,17 @@ package org.apache.flink.table.descriptors -import org.apache.flink.table.api.ValidationException -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME +import java.util +import java.util.Optional + +import org.apache.flink.table.api.{TableSchema, ValidationException} +import org.apache.flink.table.descriptors.DescriptorProperties.{toJava, toScala} +import org.apache.flink.table.descriptors.RowtimeValidator.{ROWTIME, ROWTIME_TIMESTAMPS_TYPE} import org.apache.flink.table.descriptors.SchemaValidator._ +import org.apache.flink.table.sources.RowtimeAttributeDescriptor + +import scala.collection.JavaConverters._ +import scala.collection.mutable /** * Validator for [[Schema]]. @@ -28,29 +36,39 @@ import org.apache.flink.table.descriptors.SchemaValidator._ class SchemaValidator(isStreamEnvironment: Boolean = true) extends DescriptorValidator { override def validate(properties: DescriptorProperties): Unit = { - properties.validateInt(SCHEMA_VERSION, isOptional = true, 0, Integer.MAX_VALUE) - - val names = properties.getIndexedProperty(SCHEMA, NAME) - val types = properties.getIndexedProperty(SCHEMA, TYPE) + val names = properties.getIndexedProperty(SCHEMA, SCHEMA_NAME) + val types = properties.getIndexedProperty(SCHEMA, SCHEMA_TYPE) if (names.isEmpty && types.isEmpty) { - throw new ValidationException(s"Could not find the required schema for property '$SCHEMA'.") + throw new ValidationException( + s"Could not find the required schema in property '$SCHEMA'.") } + var proctimeFound = false + for (i <- 0 until Math.max(names.size, types.size)) { - properties.validateString(s"$SCHEMA.$i.$NAME", isOptional = false, minLen = 1) - properties.validateType(s"$SCHEMA.$i.$TYPE", isOptional = false) - properties.validateString(s"$SCHEMA.$i.$FROM", isOptional = true, minLen = 1) + properties + .validateString(s"$SCHEMA.$i.$SCHEMA_NAME", isOptional = false, minLen = 1) + properties + .validateType(s"$SCHEMA.$i.$SCHEMA_TYPE", isOptional = false) + properties + .validateString(s"$SCHEMA.$i.$SCHEMA_FROM", isOptional = true, minLen = 1) // either proctime or rowtime - val proctime = s"$SCHEMA.$i.$PROCTIME" + val proctime = s"$SCHEMA.$i.$SCHEMA_PROCTIME" val rowtime = s"$SCHEMA.$i.$ROWTIME" - if (properties.contains(proctime)) { + if (properties.containsKey(proctime)) { + // check the environment if (!isStreamEnvironment) { throw new ValidationException( s"Property '$proctime' is not allowed in a batch environment.") } + // check for only one proctime attribute + else if (proctimeFound) { + throw new ValidationException("A proctime attribute must only be defined once.") + } // check proctime properties.validateBoolean(proctime, isOptional = false) + proctimeFound = properties.getBoolean(proctime) // no rowtime properties.validatePrefixExclusion(rowtime) } else if (properties.hasPrefix(rowtime)) { @@ -67,14 +85,129 @@ class SchemaValidator(isStreamEnvironment: Boolean = true) extends DescriptorVal object SchemaValidator { val SCHEMA = "schema" - val SCHEMA_VERSION = "schema.version" + val SCHEMA_NAME = "name" + val SCHEMA_TYPE = "type" + val SCHEMA_PROCTIME = "proctime" + val SCHEMA_FROM = "from" + + // utilities + + /** + * Finds the proctime attribute if defined. + */ + def deriveProctimeAttribute(properties: DescriptorProperties): Optional[String] = { + val names = properties.getIndexedProperty(SCHEMA, SCHEMA_NAME) + + for (i <- 0 until names.size) { + val isProctime = toScala( + properties.getOptionalBoolean(s"$SCHEMA.$i.$SCHEMA_PROCTIME")) + isProctime.foreach { isSet => + if (isSet) { + return toJava(names.asScala.get(s"$SCHEMA.$i.$SCHEMA_NAME")) + } + } + } + toJava(None) + } + + /** + * Finds the rowtime attributes if defined. + */ + def deriveRowtimeAttributes(properties: DescriptorProperties) + : util.List[RowtimeAttributeDescriptor] = { + + val names = properties.getIndexedProperty(SCHEMA, SCHEMA_NAME) + + var attributes = new mutable.ArrayBuffer[RowtimeAttributeDescriptor]() + + // check for rowtime in every field + for (i <- 0 until names.size) { + RowtimeValidator + .getRowtimeComponents(properties, s"$SCHEMA.$i.") + .foreach { case (extractor, strategy) => + // create descriptor + attributes += new RowtimeAttributeDescriptor( + properties.getString(s"$SCHEMA.$i.$SCHEMA_NAME"), + extractor, + strategy) + } + } + + attributes.asJava + } + + /** + * Finds a table source field mapping. + */ + def deriveFieldMapping( + properties: DescriptorProperties, + sourceSchema: Optional[TableSchema]) + : util.Map[String, String] = { + + val mapping = mutable.Map[String, String]() + + val schema = properties.getTableSchema(SCHEMA) + + // add all schema fields first for implicit mappings + schema.getColumnNames.foreach { name => + mapping.put(name, name) + } + + val names = properties.getIndexedProperty(SCHEMA, SCHEMA_NAME) + + for (i <- 0 until names.size) { + val name = properties.getString(s"$SCHEMA.$i.$SCHEMA_NAME") + toScala(properties.getOptionalString(s"$SCHEMA.$i.$SCHEMA_FROM")) match { - // per column properties + // add explicit mapping + case Some(source) => + mapping.put(name, source) - val NAME = "name" - val TYPE = "type" - val PROCTIME = "proctime" - val PROCTIME_VALUE_TRUE = "true" - val FROM = "from" + // implicit mapping or time + case None => + val isProctime = properties + .getOptionalBoolean(s"$SCHEMA.$i.$SCHEMA_PROCTIME") + .orElse(false) + val isRowtime = properties + .containsKey(s"$SCHEMA.$i.$ROWTIME_TIMESTAMPS_TYPE") + // remove proctime/rowtime from mapping + if (isProctime || isRowtime) { + mapping.remove(name) + } + // check for invalid fields + else if (toScala(sourceSchema).forall(s => !s.getColumnNames.contains(name))) { + throw new ValidationException(s"Could not map the schema field '$name' to a field " + + s"from source. Please specify the source field from which it can be derived.") + } + } + } + mapping.toMap.asJava + } + + /** + * Finds the fields that can be used for a format schema (without time attributes). + */ + def deriveFormatFields(properties: DescriptorProperties): TableSchema = { + + val builder = TableSchema.builder() + + val schema = properties.getTableSchema(SCHEMA) + + schema.getColumnNames.zip(schema.getTypes).zipWithIndex.foreach { case ((n, t), i) => + val isProctime = properties + .getOptionalBoolean(s"$SCHEMA.$i.$SCHEMA_PROCTIME") + .orElse(false) + val isRowtime = properties + .containsKey(s"$SCHEMA.$i.$ROWTIME_TIMESTAMPS_TYPE") + if (!isProctime && !isRowtime) { + // check for a aliasing + val fieldName = properties.getOptionalString(s"$SCHEMA.$i.$SCHEMA_FROM") + .orElse(n) + builder.field(fieldName, t) + } + } + + builder.build() + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Statistics.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Statistics.scala index 303728610d5b4..f87a868915b4a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Statistics.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Statistics.scala @@ -135,12 +135,12 @@ class Statistics extends Descriptor { * Internal method for properties conversion. */ final override def addProperties(properties: DescriptorProperties): Unit = { - properties.putInt(STATISTICS_VERSION, 1) + properties.putInt(STATISTICS_PROPERTY_VERSION, 1) rowCount.foreach(rc => properties.putLong(STATISTICS_ROW_COUNT, rc)) val namedStats = columnStats.map { case (name, stats) => // name should not be part of the properties key - (stats + (NAME -> name)).toMap - }.toSeq + (stats + (NAME -> name)).toMap.asJava + }.toList.asJava properties.putIndexedVariableProperties(STATISTICS_COLUMNS, namedStats) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StatisticsValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StatisticsValidator.scala index a78e42239b592..691cb218497f5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StatisticsValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StatisticsValidator.scala @@ -19,7 +19,8 @@ package org.apache.flink.table.descriptors import org.apache.flink.table.api.ValidationException -import org.apache.flink.table.descriptors.StatisticsValidator.{STATISTICS_COLUMNS, STATISTICS_ROW_COUNT, STATISTICS_VERSION, validateColumnStats} +import org.apache.flink.table.descriptors.DescriptorProperties.toScala +import org.apache.flink.table.descriptors.StatisticsValidator.{STATISTICS_COLUMNS, STATISTICS_PROPERTY_VERSION, STATISTICS_ROW_COUNT, validateColumnStats} import org.apache.flink.table.plan.stats.ColumnStats import scala.collection.mutable @@ -30,7 +31,7 @@ import scala.collection.mutable class StatisticsValidator extends DescriptorValidator { override def validate(properties: DescriptorProperties): Unit = { - properties.validateInt(STATISTICS_VERSION, isOptional = true, 0, Integer.MAX_VALUE) + properties.validateInt(STATISTICS_PROPERTY_VERSION, isOptional = true, 0, Integer.MAX_VALUE) properties.validateLong(STATISTICS_ROW_COUNT, isOptional = true, min = 0) validateColumnStats(properties, STATISTICS_COLUMNS) } @@ -38,7 +39,7 @@ class StatisticsValidator extends DescriptorValidator { object StatisticsValidator { - val STATISTICS_VERSION = "statistics.version" + val STATISTICS_PROPERTY_VERSION = "statistics.property-version" val STATISTICS_ROW_COUNT = "statistics.row-count" val STATISTICS_COLUMNS = "statistics.columns" @@ -99,16 +100,16 @@ object StatisticsValidator { val columnCount = properties.getIndexedProperty(key, NAME).size val stats = for (i <- 0 until columnCount) yield { - val name = properties.getString(s"$key.$i.$NAME").getOrElse( + val name = toScala(properties.getOptionalString(s"$key.$i.$NAME")).getOrElse( throw new ValidationException(s"Could not find name of property '$key.$i.$NAME'.")) val stats = ColumnStats( - properties.getLong(s"$key.$i.$DISTINCT_COUNT").map(v => Long.box(v)).orNull, - properties.getLong(s"$key.$i.$NULL_COUNT").map(v => Long.box(v)).orNull, - properties.getDouble(s"$key.$i.$AVG_LENGTH").map(v => Double.box(v)).orNull, - properties.getInt(s"$key.$i.$MAX_LENGTH").map(v => Int.box(v)).orNull, - properties.getDouble(s"$key.$i.$MAX_VALUE").map(v => Double.box(v)).orNull, - properties.getDouble(s"$key.$i.$MIN_VALUE").map(v => Double.box(v)).orNull + properties.getOptionalLong(s"$key.$i.$DISTINCT_COUNT").orElse(null), + properties.getOptionalLong(s"$key.$i.$NULL_COUNT").orElse(null), + properties.getOptionalDouble(s"$key.$i.$AVG_LENGTH").orElse(null), + properties.getOptionalInt(s"$key.$i.$MAX_LENGTH").orElse(null), + properties.getOptionalDouble(s"$key.$i.$MAX_VALUE").orElse(null), + properties.getOptionalDouble(s"$key.$i.$MIN_VALUE").orElse(null) ) name -> stats diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala index 5e0b42a380aeb..8f2e4736f166f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/StreamTableSourceDescriptor.scala @@ -46,7 +46,7 @@ class StreamTableSourceDescriptor(tableEnv: StreamTableEnvironment, connector: C * Searches for the specified table source, configures it accordingly, and returns it. */ def toTableSource: TableSource[_] = { - val source = TableSourceFactoryService.findTableSourceFactory(this) + val source = TableSourceFactoryService.findAndCreateTableSource(this) source match { case _: StreamTableSource[_] => source case _ => throw new TableException( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala index a49a41b5962b3..5118489d52eb0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/TableSourceDescriptor.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.descriptors +import org.apache.flink.table.descriptors.DescriptorProperties.toScala import org.apache.flink.table.descriptors.StatisticsValidator.{STATISTICS_COLUMNS, STATISTICS_ROW_COUNT, readColumnStats} import org.apache.flink.table.plan.stats.TableStats @@ -50,7 +51,7 @@ abstract class TableSourceDescriptor extends Descriptor { protected def getTableStats: Option[TableStats] = { val normalizedProps = new DescriptorProperties() addProperties(normalizedProps) - val rowCount = normalizedProps.getLong(STATISTICS_ROW_COUNT).map(v => Long.box(v)) + val rowCount = toScala(normalizedProps.getOptionalLong(STATISTICS_ROW_COUNT)) rowCount match { case Some(cnt) => val columnStats = readColumnStats(normalizedProps, STATISTICS_COLUMNS) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala index bec456543ece8..06d6bfba0d038 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSourceFactory.scala @@ -21,11 +21,12 @@ package org.apache.flink.table.sources import java.util import org.apache.flink.table.api.TableException -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_VERSION} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_PROPERTY_VERSION, CONNECTOR_TYPE} import org.apache.flink.table.descriptors.CsvValidator._ +import org.apache.flink.table.descriptors.DescriptorProperties.toScala import org.apache.flink.table.descriptors.FileSystemValidator.{CONNECTOR_PATH, CONNECTOR_TYPE_VALUE} -import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_TYPE, FORMAT_VERSION} -import org.apache.flink.table.descriptors.SchemaValidator.{SCHEMA, SCHEMA_VERSION} +import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_PROPERTY_VERSION, FORMAT_TYPE} +import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA import org.apache.flink.table.descriptors._ import org.apache.flink.types.Row @@ -38,9 +39,8 @@ class CsvTableSourceFactory extends TableSourceFactory[Row] { val context = new util.HashMap[String, String]() context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE) context.put(FORMAT_TYPE, FORMAT_TYPE_VALUE) - context.put(CONNECTOR_VERSION, "1") - context.put(FORMAT_VERSION, "1") - context.put(SCHEMA_VERSION, "1") + context.put(CONNECTOR_PROPERTY_VERSION, "1") + context.put(FORMAT_PROPERTY_VERSION, "1") context } @@ -76,33 +76,36 @@ class CsvTableSourceFactory extends TableSourceFactory[Row] { // build val csvTableSourceBuilder = new CsvTableSource.Builder - val tableSchema = params.getTableSchema(SCHEMA).get - val encodingSchema = params.getTableSchema(FORMAT_FIELDS) + val formatSchema = params.getTableSchema(FORMAT_FIELDS) + val tableSchema = params.getTableSchema(SCHEMA) // the CsvTableSource needs some rework first // for now the schema must be equal to the encoding - if (!encodingSchema.contains(tableSchema)) { + if (!formatSchema.equals(tableSchema)) { throw new TableException( "Encodings that differ from the schema are not supported yet for CsvTableSources.") } - params.getString(CONNECTOR_PATH).foreach(csvTableSourceBuilder.path) - params.getString(FORMAT_FIELD_DELIMITER).foreach(csvTableSourceBuilder.fieldDelimiter) - params.getString(FORMAT_LINE_DELIMITER).foreach(csvTableSourceBuilder.lineDelimiter) + toScala(params.getOptionalString(CONNECTOR_PATH)) + .foreach(csvTableSourceBuilder.path) + toScala(params.getOptionalString(FORMAT_FIELD_DELIMITER)) + .foreach(csvTableSourceBuilder.fieldDelimiter) + toScala(params.getOptionalString(FORMAT_LINE_DELIMITER)) + .foreach(csvTableSourceBuilder.lineDelimiter) - encodingSchema.foreach { schema => - schema.getColumnNames.zip(schema.getTypes).foreach { case (name, tpe) => - csvTableSourceBuilder.field(name, tpe) - } + formatSchema.getColumnNames.zip(formatSchema.getTypes).foreach { case (name, tpe) => + csvTableSourceBuilder.field(name, tpe) } - params.getCharacter(FORMAT_QUOTE_CHARACTER).foreach(csvTableSourceBuilder.quoteCharacter) - params.getString(FORMAT_COMMENT_PREFIX).foreach(csvTableSourceBuilder.commentPrefix) - params.getBoolean(FORMAT_IGNORE_FIRST_LINE).foreach { flag => + toScala(params.getOptionalCharacter(FORMAT_QUOTE_CHARACTER)) + .foreach(csvTableSourceBuilder.quoteCharacter) + toScala(params.getOptionalString(FORMAT_COMMENT_PREFIX)) + .foreach(csvTableSourceBuilder.commentPrefix) + toScala(params.getOptionalBoolean(FORMAT_IGNORE_FIRST_LINE)).foreach { flag => if (flag) { csvTableSourceBuilder.ignoreFirstLine() } } - params.getBoolean(FORMAT_IGNORE_PARSE_ERRORS).foreach { flag => + toScala(params.getOptionalBoolean(FORMAT_IGNORE_PARSE_ERRORS)).foreach { flag => if (flag) { csvTableSourceBuilder.ignoreParseErrors() } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala index f42d765ebca4c..e5f696503279a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactory.scala @@ -41,10 +41,10 @@ trait TableSourceFactory[T] { * - connector.type * - format.type * - * Specified versions allow the framework to provide backwards compatible properties in case of - * string format changes: - * - connector.version - * - format.version + * Specified property versions allow the framework to provide backwards compatible properties + * in case of string format changes: + * - connector.property-version + * - format.property-version * * An empty context means that the factory matches for all requests. */ @@ -61,7 +61,8 @@ trait TableSourceFactory[T] { * - format.fields.#.type * - format.fields.#.name * - * Note: Use "#" to denote an array of values where "#" represents one or more digits. + * Note: Use "#" to denote an array of values where "#" represents one or more digits. Property + * versions like "format.property-version" must not be part of the supported properties. */ def supportedProperties(): util.List[String] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala index 1e8e83691a249..877cb7b5f3912 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala @@ -21,12 +21,10 @@ package org.apache.flink.table.sources import java.util.{ServiceConfigurationError, ServiceLoader} import org.apache.flink.table.api.{AmbiguousTableSourceException, NoMatchingTableSourceException, TableException, ValidationException} -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION -import org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_VERSION -import org.apache.flink.table.descriptors.MetadataValidator.METADATA_VERSION -import org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_VERSION -import org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_VERSION -import org.apache.flink.table.descriptors.StatisticsValidator.STATISTICS_VERSION +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION +import org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_PROPERTY_VERSION +import org.apache.flink.table.descriptors.MetadataValidator.METADATA_PROPERTY_VERSION +import org.apache.flink.table.descriptors.StatisticsValidator.STATISTICS_PROPERTY_VERSION import org.apache.flink.table.descriptors._ import org.apache.flink.table.util.Logging @@ -40,13 +38,13 @@ object TableSourceFactoryService extends Logging { private lazy val loader = ServiceLoader.load(classOf[TableSourceFactory[_]]) - def findTableSourceFactory(descriptor: TableSourceDescriptor): TableSource[_] = { + def findAndCreateTableSource(descriptor: TableSourceDescriptor): TableSource[_] = { val properties = new DescriptorProperties() descriptor.addProperties(properties) - findTableSourceFactory(properties.asMap) + findAndCreateTableSource(properties.asMap.asScala.toMap) } - def findTableSourceFactory(properties: Map[String, String]): TableSource[_] = { + def findAndCreateTableSource(properties: Map[String, String]): TableSource[_] = { var matchingFactory: Option[(TableSourceFactory[_], Seq[String])] = None try { val iter = loader.iterator() @@ -73,12 +71,10 @@ object TableSourceFactoryService extends Logging { plainContext ++= requiredContext // we remove the versions for now until we have the first backwards compatibility case // with the version we can provide mappings in case the format changes - plainContext.remove(CONNECTOR_VERSION) - plainContext.remove(FORMAT_VERSION) - plainContext.remove(SCHEMA_VERSION) - plainContext.remove(ROWTIME_VERSION) - plainContext.remove(METADATA_VERSION) - plainContext.remove(STATISTICS_VERSION) + plainContext.remove(CONNECTOR_PROPERTY_VERSION) + plainContext.remove(FORMAT_PROPERTY_VERSION) + plainContext.remove(METADATA_PROPERTY_VERSION) + plainContext.remove(STATISTICS_PROPERTY_VERSION) // check if required context is met if (plainContext.forall(e => properties.contains(e._1) && properties(e._1) == e._2)) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/tsextractors/StreamRecordTimestamp.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/tsextractors/StreamRecordTimestamp.scala index 329f790b9607c..fcbd63f1bb204 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/tsextractors/StreamRecordTimestamp.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/tsextractors/StreamRecordTimestamp.scala @@ -27,7 +27,7 @@ import org.apache.flink.table.expressions.{Expression, ResolvedFieldReference} * * Note: This extractor only works for StreamTableSources. */ -class StreamRecordTimestamp extends TimestampExtractor { +final class StreamRecordTimestamp extends TimestampExtractor { /** No argument fields required. */ override def getArgumentFields: Array[String] = Array() @@ -42,5 +42,8 @@ class StreamRecordTimestamp extends TimestampExtractor { override def getExpression(fieldAccesses: Array[ResolvedFieldReference]): Expression = { org.apache.flink.table.expressions.StreamRecordTimestamp() } +} +object StreamRecordTimestamp { + val INSTANCE: StreamRecordTimestamp = new StreamRecordTimestamp } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/CsvTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/CsvTest.scala index 15cf13bd72279..85778ca02b105 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/CsvTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/CsvTest.scala @@ -18,16 +18,36 @@ package org.apache.flink.table.descriptors +import java.util + import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.TypeExtractor import org.apache.flink.table.api.{TableSchema, Types, ValidationException} import org.junit.Test +import scala.collection.JavaConverters._ + class CsvTest extends DescriptorTestBase { - @Test - def testCsv(): Unit = { - val desc = Csv() + @Test(expected = classOf[ValidationException]) + def testInvalidType(): Unit = { + addPropertyAndVerify(descriptors().get(0), "format.fields.0.type", "WHATEVER") + } + + @Test(expected = classOf[ValidationException]) + def testInvalidField(): Unit = { + addPropertyAndVerify(descriptors().get(0), "format.fields.10.name", "WHATEVER") + } + + @Test(expected = classOf[ValidationException]) + def testInvalidQuoteCharacter(): Unit = { + addPropertyAndVerify(descriptors().get(0), "format.quote-character", "qq") + } + + // ---------------------------------------------------------------------------------------------- + + override def descriptors(): util.List[Descriptor] = { + val desc1 = Csv() .field("field1", "STRING") .field("field2", Types.SQL_TIMESTAMP) .field("field3", TypeExtractor.createTypeInfo(classOf[Class[_]])) @@ -35,9 +55,21 @@ class CsvTest extends DescriptorTestBase { Array[String]("test", "row"), Array[TypeInformation[_]](Types.INT, Types.STRING))) .lineDelimiter("^") - val expected = Seq( + + val desc2 = Csv() + .schema(new TableSchema( + Array[String]("test", "row"), + Array[TypeInformation[_]](Types.INT, Types.STRING))) + .quoteCharacter('#') + .ignoreFirstLine() + + util.Arrays.asList(desc1, desc2) + } + + override def properties(): util.List[util.Map[String, String]] = { + val props1 = Map( "format.type" -> "csv", - "format.version" -> "1", + "format.property-version" -> "1", "format.fields.0.name" -> "field1", "format.fields.0.type" -> "STRING", "format.fields.1.name" -> "field2", @@ -47,53 +79,18 @@ class CsvTest extends DescriptorTestBase { "format.fields.3.name" -> "field4", "format.fields.3.type" -> "ROW(test INT, row VARCHAR)", "format.line-delimiter" -> "^") - verifyProperties(desc, expected) - } - @Test - def testCsvTableSchema(): Unit = { - val desc = Csv() - .schema(new TableSchema( - Array[String]("test", "row"), - Array[TypeInformation[_]](Types.INT, Types.STRING))) - .quoteCharacter('#') - .ignoreFirstLine() - val expected = Seq( + val props2 = Map( "format.type" -> "csv", - "format.version" -> "1", + "format.property-version" -> "1", "format.fields.0.name" -> "test", "format.fields.0.type" -> "INT", "format.fields.1.name" -> "row", "format.fields.1.type" -> "VARCHAR", "format.quote-character" -> "#", "format.ignore-first-line" -> "true") - verifyProperties(desc, expected) - } - @Test(expected = classOf[ValidationException]) - def testInvalidType(): Unit = { - verifyInvalidProperty("format.fields.0.type", "WHATEVER") - } - - @Test(expected = classOf[ValidationException]) - def testInvalidField(): Unit = { - verifyInvalidProperty("format.fields.10.name", "WHATEVER") - } - - @Test(expected = classOf[ValidationException]) - def testInvalidQuoteCharacter(): Unit = { - verifyInvalidProperty("format.quote-character", "qq") - } - - override def descriptor(): Descriptor = { - Csv() - .field("field1", "STRING") - .field("field2", Types.SQL_TIMESTAMP) - .field("field3", TypeExtractor.createTypeInfo(classOf[Class[_]])) - .field("field4", Types.ROW( - Array[String]("test", "row"), - Array[TypeInformation[_]](Types.INT, Types.STRING))) - .lineDelimiter("^") + util.Arrays.asList(props1.asJava, props2.asJava) } override def validator(): DescriptorValidator = { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorTestBase.scala index 3a59c9be5133d..7a98b0be23fe5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/DescriptorTestBase.scala @@ -18,37 +18,84 @@ package org.apache.flink.table.descriptors +import org.apache.flink.util.Preconditions import org.junit.Assert.assertEquals +import org.junit.Test + +import scala.collection.JavaConverters._ abstract class DescriptorTestBase { /** - * Returns a valid descriptor. + * Returns a set of valid descriptors. + * This method is implemented in both Scala and Java. */ - def descriptor(): Descriptor + def descriptors(): java.util.List[Descriptor] /** - * Returns a validator that can validate this descriptor. + * Returns a set of properties for each valid descriptor. + * This code is implemented in both Scala and Java. + */ + def properties(): java.util.List[java.util.Map[String, String]] + + /** + * Returns a validator that can validate all valid descriptors. */ def validator(): DescriptorValidator - def verifyProperties(descriptor: Descriptor, expected: Seq[(String, String)]): Unit = { + @Test + def testValidation(): Unit = { + val d = descriptors().asScala + val p = properties().asScala + + Preconditions.checkArgument(d.length == p.length) + + d.zip(p).foreach { case (desc, props) => + verifyProperties(desc, props.asScala.toMap) + } + } + + def verifyProperties(descriptor: Descriptor, expected: Map[String, String]): Unit = { val normProps = new DescriptorProperties descriptor.addProperties(normProps) - assertEquals(expected.toMap, normProps.asMap) + + // test produced properties + assertEquals(expected, normProps.asMap.asScala.toMap) + + // test validation logic + validator().validate(normProps) } - def verifyInvalidProperty(property: String, invalidValue: String): Unit = { + def addPropertyAndVerify( + descriptor: Descriptor, + property: String, + invalidValue: String): Unit = { val properties = new DescriptorProperties - descriptor().addProperties(properties) + descriptor.addProperties(properties) properties.unsafePut(property, invalidValue) validator().validate(properties) } - def verifyMissingProperty(removeProperty: String): Unit = { + def removePropertyAndVerify(descriptor: Descriptor, removeProperty: String): Unit = { val properties = new DescriptorProperties - descriptor().addProperties(properties) + descriptor.addProperties(properties) properties.unsafeRemove(removeProperty) validator().validate(properties) } } + +class TestTableSourceDescriptor(connector: ConnectorDescriptor) + extends TableSourceDescriptor { + + this.connectorDescriptor = Some(connector) + + def addFormat(format: FormatDescriptor): TestTableSourceDescriptor = { + this.formatDescriptor = Some(format) + this + } + + def addSchema(schema: Schema): TestTableSourceDescriptor = { + this.schemaDescriptor = Some(schema) + this + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala index 3452e8d31a1cf..1162694a01dc7 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/FileSystemTest.scala @@ -18,36 +18,41 @@ package org.apache.flink.table.descriptors +import java.util + import org.apache.flink.table.api.ValidationException import org.junit.Test -class FileSystemTest extends DescriptorTestBase { +import scala.collection.JavaConverters._ - @Test - def testFileSystem(): Unit = { - val desc = FileSystem().path("/myfile") - val expected = Seq( - "connector.type" -> "filesystem", - "connector.version" -> "1", - "connector.path" -> "/myfile") - verifyProperties(desc, expected) - } +class FileSystemTest extends DescriptorTestBase { @Test(expected = classOf[ValidationException]) def testInvalidPath(): Unit = { - verifyInvalidProperty("connector.path", "") + addPropertyAndVerify(descriptors().get(0), "connector.path", "") } @Test(expected = classOf[ValidationException]) def testMissingPath(): Unit = { - verifyMissingProperty("connector.path") + removePropertyAndVerify(descriptors().get(0), "connector.path") } - override def descriptor(): Descriptor = { - FileSystem().path("/myfile") + // ---------------------------------------------------------------------------------------------- + + override def descriptors(): util.List[Descriptor] = { + util.Arrays.asList(FileSystem().path("/myfile")) } override def validator(): DescriptorValidator = { new FileSystemValidator() } + + override def properties(): util.List[util.Map[String, String]] = { + val desc = Map( + "connector.type" -> "filesystem", + "connector.property-version" -> "1", + "connector.path" -> "/myfile") + + util.Arrays.asList(desc.asJava) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/JsonTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/JsonTest.scala deleted file mode 100644 index 756ca231bb683..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/JsonTest.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.table.descriptors - -import org.apache.flink.table.api.ValidationException -import org.junit.Test - -class JsonTest extends DescriptorTestBase { - - @Test - def testJson(): Unit = { - val schema = - """ - |{ - | "title": "Person", - | "type": "object", - | "properties": { - | "firstName": { - | "type": "string" - | }, - | "lastName": { - | "type": "string" - | }, - | "age": { - | "description": "Age in years", - | "type": "integer", - | "minimum": 0 - | } - | }, - | "required": ["firstName", "lastName"] - |} - |""".stripMargin - val desc = Json() - .schema(schema) - .failOnMissingField(true) - val expected = Seq( - "format.type" -> "json", - "format.version" -> "1", - "format.schema-string" -> schema, - "format.fail-on-missing-field" -> "true") - verifyProperties(desc, expected) - } - - @Test(expected = classOf[ValidationException]) - def testInvalidMissingField(): Unit = { - verifyInvalidProperty("format.fail-on-missing-field", "DDD") - } - - @Test(expected = classOf[ValidationException]) - def testMissingSchema(): Unit = { - verifyMissingProperty("format.schema-string") - } - - override def descriptor(): Descriptor = { - Json().schema("test") - } - - override def validator(): DescriptorValidator = { - new JsonValidator() - } -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/MetadataTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/MetadataTest.scala index a1854ce888046..64965b0fa9eea 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/MetadataTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/MetadataTest.scala @@ -18,38 +18,42 @@ package org.apache.flink.table.descriptors +import java.util + import org.apache.flink.table.api.ValidationException import org.junit.Test -class MetadataTest extends DescriptorTestBase { +import scala.collection.JavaConverters._ - @Test - def testMetadata(): Unit = { - val desc = Metadata() - .comment("Some additional comment") - .creationTime(123L) - .lastAccessTime(12020202L) - val expected = Seq( - "metadata.comment" -> "Some additional comment", - "metadata.creation-time" -> "123", - "metadata.last-access-time" -> "12020202" - ) - verifyProperties(desc, expected) - } +class MetadataTest extends DescriptorTestBase { @Test(expected = classOf[ValidationException]) def testInvalidCreationTime(): Unit = { - verifyInvalidProperty("metadata.creation-time", "dfghj") + addPropertyAndVerify(descriptors().get(0), "metadata.creation-time", "dfghj") } - override def descriptor(): Descriptor = { - Metadata() + // ---------------------------------------------------------------------------------------------- + + override def descriptors(): util.List[Descriptor] = { + val desc = Metadata() .comment("Some additional comment") .creationTime(123L) .lastAccessTime(12020202L) + + util.Arrays.asList(desc) } override def validator(): DescriptorValidator = { new MetadataValidator() } + + override def properties(): util.List[util.Map[String, String]] = { + val props = Map( + "metadata.comment" -> "Some additional comment", + "metadata.creation-time" -> "123", + "metadata.last-access-time" -> "12020202" + ) + + util.Arrays.asList(props.asJava) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala index 80050fc26d4b6..7968b481db343 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala @@ -18,6 +18,8 @@ package org.apache.flink.table.descriptors +import java.util + import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.table.api.ValidationException import org.apache.flink.table.descriptors.RowtimeTest.CustomAssigner @@ -25,41 +27,58 @@ import org.apache.flink.table.sources.wmstrategies.PunctuatedWatermarkAssigner import org.apache.flink.types.Row import org.junit.Test -class RowtimeTest extends DescriptorTestBase { +import scala.collection.JavaConverters._ - @Test - def testRowtime(): Unit = { - val desc = Rowtime() - .timestampsFromField("otherField") - .watermarksPeriodicBounding(1000L) - val expected = Seq( - "rowtime.0.version" -> "1", - "rowtime.0.timestamps.type" -> "from-field", - "rowtime.0.timestamps.from" -> "otherField", - "rowtime.0.watermarks.type" -> "periodic-bounding", - "rowtime.0.watermarks.delay" -> "1000" - ) - verifyProperties(desc, expected) - } +class RowtimeTest extends DescriptorTestBase { @Test(expected = classOf[ValidationException]) def testInvalidWatermarkType(): Unit = { - verifyInvalidProperty("rowtime.0.watermarks.type", "xxx") + addPropertyAndVerify(descriptors().get(0), "rowtime.watermarks.type", "xxx") } @Test(expected = classOf[ValidationException]) def testMissingWatermarkClass(): Unit = { - verifyMissingProperty("rowtime.0.watermarks.class") + removePropertyAndVerify(descriptors().get(1), "rowtime.watermarks.class") } - override def descriptor(): Descriptor = { - Rowtime() + // ---------------------------------------------------------------------------------------------- + + override def descriptors(): util.List[Descriptor] = { + val desc1 = Rowtime() + .timestampsFromField("otherField") + .watermarksPeriodicBounding(1000L) + + val desc2 = Rowtime() .timestampsFromSource() .watermarksFromStrategy(new CustomAssigner()) + + util.Arrays.asList(desc1, desc2) } override def validator(): DescriptorValidator = { - new RowtimeValidator("rowtime.0.") + new RowtimeValidator() + } + + override def properties(): util.List[util.Map[String, String]] = { + val props1 = Map( + "rowtime.timestamps.type" -> "from-field", + "rowtime.timestamps.from" -> "otherField", + "rowtime.watermarks.type" -> "periodic-bounded", + "rowtime.watermarks.delay" -> "1000" + ) + + val props2 = Map( + "rowtime.timestamps.type" -> "from-source", + "rowtime.watermarks.type" -> "custom", + "rowtime.watermarks.class" -> "org.apache.flink.table.descriptors.RowtimeTest$CustomAssigner", + "rowtime.watermarks.serialized" -> ("rO0ABXNyAD1vcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmRlc2NyaX" + + "B0b3JzLlJvd3RpbWVUZXN0JEN1c3RvbUFzc2lnbmVyeDcuDvfbu0kCAAB4cgBHb3JnLmFwYWNoZS5mbGluay" + + "50YWJsZS5zb3VyY2VzLndtc3RyYXRlZ2llcy5QdW5jdHVhdGVkV2F0ZXJtYXJrQXNzaWduZXKBUc57oaWu9A" + + "IAAHhyAD1vcmcuYXBhY2hlLmZsaW5rLnRhYmxlLnNvdXJjZXMud21zdHJhdGVnaWVzLldhdGVybWFya1N0cm" + + "F0ZWd5mB_uSxDZ8-MCAAB4cA") + ) + + util.Arrays.asList(props1.asJava, props2.asJava) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaTest.scala index f663a96be1545..589ec4f582b3d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaTest.scala @@ -18,21 +18,54 @@ package org.apache.flink.table.descriptors +import java.util + import org.apache.flink.table.api.{Types, ValidationException} import org.junit.Test +import scala.collection.JavaConverters._ + class SchemaTest extends DescriptorTestBase { - @Test - def testSchema(): Unit = { - val desc = Schema() + @Test(expected = classOf[ValidationException]) + def testInvalidType(): Unit = { + addPropertyAndVerify( + descriptors().get(0), + "schema.1.type", "dfghj") + } + + @Test(expected = classOf[ValidationException]) + def testBothRowtimeAndProctime(): Unit = { + addPropertyAndVerify( + descriptors().get(0), + "schema.2.rowtime.watermarks.type", "from-source") + } + + // ---------------------------------------------------------------------------------------------- + + override def descriptors(): util.List[Descriptor] = { + val desc1 = Schema() .field("myField", Types.BOOLEAN) .field("otherField", "VARCHAR").from("csvField") .field("p", Types.SQL_TIMESTAMP).proctime() .field("r", Types.SQL_TIMESTAMP).rowtime( Rowtime().timestampsFromSource().watermarksFromSource()) - val expected = Seq( - "schema.version" -> "1", + + val desc2 = Schema() + .field("myField", Types.BOOLEAN) + .field("otherField", "VARCHAR").from("csvField") + .field("p", Types.SQL_TIMESTAMP).proctime() + .field("r", Types.SQL_TIMESTAMP) + + util.Arrays.asList(desc1, desc2) + } + + override def validator(): DescriptorValidator = { + new SchemaValidator(isStreamEnvironment = true) + } + + override def properties(): util.List[util.Map[String, String]] = { + val props1 = Map( "schema.0.name" -> "myField", "schema.0.type" -> "BOOLEAN", "schema.1.name" -> "otherField", @@ -43,34 +76,23 @@ class SchemaTest extends DescriptorTestBase { "schema.2.proctime" -> "true", "schema.3.name" -> "r", "schema.3.type" -> "TIMESTAMP", - "schema.3.rowtime.0.version" -> "1", - "schema.3.rowtime.0.watermarks.type" -> "from-source", - "schema.3.rowtime.0.timestamps.type" -> "from-source" + "schema.3.rowtime.watermarks.type" -> "from-source", + "schema.3.rowtime.timestamps.type" -> "from-source" ) - verifyProperties(desc, expected) - } - @Test(expected = classOf[ValidationException]) - def testInvalidType(): Unit = { - verifyInvalidProperty("schema.1.type", "dfghj") - } - - @Test(expected = classOf[ValidationException]) - def testBothRowtimeAndProctime(): Unit = { - verifyInvalidProperty("schema.2.rowtime.0.version", "1") - verifyInvalidProperty("schema.2.rowtime.0.watermarks.type", "from-source") - verifyInvalidProperty("schema.2.rowtime.0.timestamps.type", "from-source") - } - - override def descriptor(): Descriptor = { - Schema() - .field("myField", Types.BOOLEAN) - .field("otherField", "VARCHAR").from("csvField") - .field("p", Types.SQL_TIMESTAMP).proctime() - .field("r", Types.SQL_TIMESTAMP) - } + val props2 = Map( + "schema.0.name" -> "myField", + "schema.0.type" -> "BOOLEAN", + "schema.1.name" -> "otherField", + "schema.1.type" -> "VARCHAR", + "schema.1.from" -> "csvField", + "schema.2.name" -> "p", + "schema.2.type" -> "TIMESTAMP", + "schema.2.proctime" -> "true", + "schema.3.name" -> "r", + "schema.3.type" -> "TIMESTAMP" + ) - override def validator(): DescriptorValidator = { - new SchemaValidator(isStreamEnvironment = true) + util.Arrays.asList(props1.asJava, props2.asJava) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala new file mode 100644 index 0000000000000..ba05dfff2074d --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala @@ -0,0 +1,76 @@ +/* + * 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.table.descriptors + +import java.util.Optional + +import org.apache.flink.table.api.{TableSchema, Types} +import org.apache.flink.table.sources.tsextractors.StreamRecordTimestamp +import org.apache.flink.table.sources.wmstrategies.PreserveWatermarks +import org.junit.Assert.{assertEquals, assertTrue} +import org.junit.Test + +import scala.collection.JavaConverters._ + +/** + * Tests for [[SchemaValidator]]. + */ +class SchemaValidatorTest { + + @Test + def testSchema(): Unit = { + val desc1 = Schema() + .field("otherField", Types.STRING).from("csvField") + .field("abcField", Types.STRING) + .field("p", Types.SQL_TIMESTAMP).proctime() + .field("r", Types.SQL_TIMESTAMP).rowtime( + Rowtime().timestampsFromSource().watermarksFromSource()) + val props = new DescriptorProperties() + desc1.addProperties(props) + + val inputSchema = TableSchema.builder() + .field("csvField", Types.STRING) + .field("abcField", Types.STRING) + .field("myField", Types.BOOLEAN) + .build() + + // test proctime + assertEquals(Optional.of("p"), SchemaValidator.deriveProctimeAttribute(props)) + + // test rowtime + val rowtime = SchemaValidator.deriveRowtimeAttributes(props).get(0) + assertEquals("r", rowtime.getAttributeName) + assertTrue(rowtime.getTimestampExtractor.isInstanceOf[StreamRecordTimestamp]) + assertTrue(rowtime.getWatermarkStrategy.isInstanceOf[PreserveWatermarks]) + + // test field mapping + val expectedMapping = Map("otherField" -> "csvField", "abcField" -> "abcField").asJava + assertEquals( + expectedMapping, + SchemaValidator.deriveFieldMapping(props, Optional.of(inputSchema))) + + // test field format + val formatSchema = SchemaValidator.deriveFormatFields(props) + val expectedFormatSchema = TableSchema.builder() + .field("csvField", Types.STRING) // aliased + .field("abcField", Types.STRING) + .build() + assertEquals(expectedFormatSchema, formatSchema) + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/StatisticsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/StatisticsTest.scala index 3b248b47b7615..2def0c317a498 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/StatisticsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/StatisticsTest.scala @@ -24,17 +24,44 @@ import org.apache.flink.table.api.ValidationException import org.apache.flink.table.plan.stats.{ColumnStats, TableStats} import org.junit.Test +import scala.collection.JavaConverters._ + class StatisticsTest extends DescriptorTestBase { - @Test - def testStatistics(): Unit = { - val desc = Statistics() + @Test(expected = classOf[ValidationException]) + def testInvalidRowCount(): Unit = { + addPropertyAndVerify(descriptors().get(0), "statistics.row-count", "abx") + } + + @Test(expected = classOf[ValidationException]) + def testMissingName(): Unit = { + removePropertyAndVerify(descriptors().get(0), "statistics.columns.0.name") + } + + // ---------------------------------------------------------------------------------------------- + + override def descriptors(): util.List[Descriptor] = { + val desc1 = Statistics() .rowCount(1000L) .columnStats("a", ColumnStats(1L, 2L, 3.0, 4, 5, 6)) .columnAvgLength("b", 42.0) .columnNullCount("a", 300) - val expected = Seq( - "statistics.version" -> "1", + + val map = new util.HashMap[String, ColumnStats]() + map.put("a", ColumnStats(null, 2L, 3.0, null, 5, 6)) + val desc2 = Statistics() + .tableStats(TableStats(32L, map)) + + util.Arrays.asList(desc1, desc2) + } + + override def validator(): DescriptorValidator = { + new StatisticsValidator() + } + + override def properties(): util.List[util.Map[String, String]] = { + val props1 = Map( + "statistics.property-version" -> "1", "statistics.row-count" -> "1000", "statistics.columns.0.name" -> "a", "statistics.columns.0.distinct-count" -> "1", @@ -46,17 +73,9 @@ class StatisticsTest extends DescriptorTestBase { "statistics.columns.1.name" -> "b", "statistics.columns.1.avg-length" -> "42.0" ) - verifyProperties(desc, expected) - } - @Test - def testStatisticsTableStats(): Unit = { - val map = new util.HashMap[String, ColumnStats]() - map.put("a", ColumnStats(null, 2L, 3.0, null, 5, 6)) - val desc = Statistics() - .tableStats(TableStats(32L, map)) - val expected = Seq( - "statistics.version" -> "1", + val props2 = Map( + "statistics.property-version" -> "1", "statistics.row-count" -> "32", "statistics.columns.0.name" -> "a", "statistics.columns.0.null-count" -> "2", @@ -64,28 +83,7 @@ class StatisticsTest extends DescriptorTestBase { "statistics.columns.0.max-value" -> "5", "statistics.columns.0.min-value" -> "6" ) - verifyProperties(desc, expected) - } - @Test(expected = classOf[ValidationException]) - def testInvalidRowCount(): Unit = { - verifyInvalidProperty("statistics.row-count", "abx") - } - - @Test(expected = classOf[ValidationException]) - def testMissingName(): Unit = { - verifyMissingProperty("statistics.columns.0.name") - } - - override def descriptor(): Descriptor = { - Statistics() - .rowCount(1000L) - .columnStats("a", ColumnStats(1L, 2L, 3.0, 4, 5, 6)) - .columnAvgLength("b", 42.0) - .columnNullCount("a", 300) - } - - override def validator(): DescriptorValidator = { - new StatisticsValidator() + util.Arrays.asList(props1.asJava, props2.asJava) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala index 5e9b5a24c022b..279e9a41344b1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TableSourceFactoryServiceTest.scala @@ -19,8 +19,8 @@ package org.apache.flink.table.sources import org.apache.flink.table.api.{NoMatchingTableSourceException, TableException, ValidationException} -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_VERSION} -import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_TYPE, FORMAT_VERSION} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_PROPERTY_VERSION} +import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_TYPE, FORMAT_PROPERTY_VERSION} import org.junit.Assert.assertTrue import org.junit.Test @@ -31,44 +31,44 @@ class TableSourceFactoryServiceTest { @Test def testValidProperties(): Unit = { val props = properties() - assertTrue(TableSourceFactoryService.findTableSourceFactory(props.toMap) != null) + assertTrue(TableSourceFactoryService.findAndCreateTableSource(props.toMap) != null) } @Test(expected = classOf[NoMatchingTableSourceException]) def testInvalidContext(): Unit = { val props = properties() props.put(CONNECTOR_TYPE, "FAIL") - TableSourceFactoryService.findTableSourceFactory(props.toMap) + TableSourceFactoryService.findAndCreateTableSource(props.toMap) } @Test def testDifferentContextVersion(): Unit = { val props = properties() - props.put(CONNECTOR_VERSION, "2") + props.put(CONNECTOR_PROPERTY_VERSION, "2") // the table source should still be found - assertTrue(TableSourceFactoryService.findTableSourceFactory(props.toMap) != null) + assertTrue(TableSourceFactoryService.findAndCreateTableSource(props.toMap) != null) } @Test(expected = classOf[ValidationException]) def testUnsupportedProperty(): Unit = { val props = properties() props.put("format.path_new", "/new/path") - TableSourceFactoryService.findTableSourceFactory(props.toMap) + TableSourceFactoryService.findAndCreateTableSource(props.toMap) } @Test(expected = classOf[TableException]) def testFailingFactory(): Unit = { val props = properties() props.put("failing", "true") - TableSourceFactoryService.findTableSourceFactory(props.toMap) + TableSourceFactoryService.findAndCreateTableSource(props.toMap) } private def properties(): mutable.Map[String, String] = { val properties = mutable.Map[String, String]() properties.put(CONNECTOR_TYPE, "test") properties.put(FORMAT_TYPE, "test") - properties.put(CONNECTOR_VERSION, "1") - properties.put(FORMAT_VERSION, "1") + properties.put(CONNECTOR_PROPERTY_VERSION, "1") + properties.put(FORMAT_PROPERTY_VERSION, "1") properties.put("format.path", "/path/to/target") properties.put("schema.0.name", "a") properties.put("schema.1.name", "b") diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestTableSourceFactory.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestTableSourceFactory.scala index ae75f99e14d63..ee3d637b7b142 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestTableSourceFactory.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/sources/TestTableSourceFactory.scala @@ -22,8 +22,8 @@ import java.util import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api.TableSchema -import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_VERSION} -import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_TYPE, FORMAT_VERSION} +import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.{CONNECTOR_TYPE, CONNECTOR_PROPERTY_VERSION} +import org.apache.flink.table.descriptors.FormatDescriptorValidator.{FORMAT_TYPE, FORMAT_PROPERTY_VERSION} import org.apache.flink.types.Row class TestTableSourceFactory extends TableSourceFactory[Row] { @@ -32,8 +32,8 @@ class TestTableSourceFactory extends TableSourceFactory[Row] { val context = new util.HashMap[String, String]() context.put(CONNECTOR_TYPE, "test") context.put(FORMAT_TYPE, "test") - context.put(CONNECTOR_VERSION, "1") - context.put(FORMAT_VERSION, "1") + context.put(CONNECTOR_PROPERTY_VERSION, "1") + context.put(FORMAT_PROPERTY_VERSION, "1") context } From 23358ff87003fd6603c0ca19bc37f31944d2c494 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Mon, 26 Feb 2018 16:41:24 +0100 Subject: [PATCH 006/268] [FLINK-8791] [docs] Fix documentation about configuring dependencies --- docs/dev/linking.md | 96 ------- docs/dev/linking_with_flink.md | 146 ----------- docs/redirects/linking_with_flink.md | 25 ++ .../linking_with_optional_modules.md | 25 ++ docs/start/dependencies.md | 244 ++++++++++++++++++ 5 files changed, 294 insertions(+), 242 deletions(-) delete mode 100644 docs/dev/linking.md delete mode 100644 docs/dev/linking_with_flink.md create mode 100644 docs/redirects/linking_with_flink.md create mode 100644 docs/redirects/linking_with_optional_modules.md create mode 100644 docs/start/dependencies.md diff --git a/docs/dev/linking.md b/docs/dev/linking.md deleted file mode 100644 index 78ef54494831f..0000000000000 --- a/docs/dev/linking.md +++ /dev/null @@ -1,96 +0,0 @@ ---- -nav-title: "Linking with Optional Modules" -title: "Linking with modules not contained in the binary distribution" -nav-parent_id: start -nav-pos: 10 ---- - - -The binary distribution contains jar packages in the `lib` folder that are automatically -provided to the classpath of your distributed programs. Almost all of Flink classes are -located there with a few exceptions, for example the streaming connectors and some freshly -added modules. To run code depending on these modules you need to make them accessible -during runtime, for which we suggest two options: - -1. Either copy the required jar files to the `lib` folder onto all of your TaskManagers. -Note that you have to restart your TaskManagers after this. -2. Or package them with your code. - -The latter version is recommended as it respects the classloader management in Flink. - -### Packaging dependencies with your usercode with Maven - -To provide these dependencies not included by Flink we suggest two options with Maven. - -1. The maven assembly plugin builds a so-called uber-jar (executable jar) containing all your dependencies. -The assembly configuration is straight-forward, but the resulting jar might become bulky. -See [maven-assembly-plugin](http://maven.apache.org/plugins/maven-assembly-plugin/usage.html) for further information. -2. The maven unpack plugin unpacks the relevant parts of the dependencies and -then packages it with your code. - -Using the latter approach in order to bundle the Kafka connector, `flink-connector-kafka` -you would need to add the classes from both the connector and the Kafka API itself. Add -the following to your plugins section. - -~~~xml - - org.apache.maven.plugins - maven-dependency-plugin - 2.9 - - - unpack - - prepare-package - - unpack - - - - - - org.apache.flink - flink-connector-kafka - {{ site.version }} - jar - false - ${project.build.directory}/classes - org/apache/flink/** - - - - org.apache.kafka - kafka_ - - jar - false - ${project.build.directory}/classes - kafka/** - - - - - - -~~~ - -Now when running `mvn clean package` the produced jar includes the required dependencies. - -{% top %} diff --git a/docs/dev/linking_with_flink.md b/docs/dev/linking_with_flink.md deleted file mode 100644 index f2380b23a07f6..0000000000000 --- a/docs/dev/linking_with_flink.md +++ /dev/null @@ -1,146 +0,0 @@ ---- -title: "Linking with Flink" -nav-parent_id: start -nav-pos: 2 ---- - - -To write programs with Flink, you need to include the Flink library corresponding to -your programming language in your project. - -The simplest way to do this is to use one of the quickstart scripts: either for -[Java]({{ site.baseurl }}/quickstart/java_api_quickstart.html) or for [Scala]({{ site.baseurl }}/quickstart/scala_api_quickstart.html). They -create a blank project from a template (a Maven Archetype), which sets up everything for you. To -manually create the project, you can use the archetype and create a project by calling: - -

    -
    -{% highlight bash %} -mvn archetype:generate \ - -DarchetypeGroupId=org.apache.flink \ - -DarchetypeArtifactId=flink-quickstart-java \ - -DarchetypeVersion={{site.version }} -{% endhighlight %} -
    -
    -{% highlight bash %} -mvn archetype:generate \ - -DarchetypeGroupId=org.apache.flink \ - -DarchetypeArtifactId=flink-quickstart-scala \ - -DarchetypeVersion={{site.version }} -{% endhighlight %} -
    -
    - -The archetypes are working for stable releases and preview versions (`-SNAPSHOT`). - -If you want to add Flink to an existing Maven project, add the following entry to your -*dependencies* section in the *pom.xml* file of your project: - -
    -
    -{% highlight xml %} - - - org.apache.flink - flink-streaming-java{{ site.scala_version_suffix }} - {{site.version }} - - - - org.apache.flink - flink-java - {{site.version }} - - - org.apache.flink - flink-clients{{ site.scala_version_suffix }} - {{site.version }} - -{% endhighlight %} -
    -
    -{% highlight xml %} - - - org.apache.flink - flink-streaming-scala{{ site.scala_version_suffix }} - {{site.version }} - - - - org.apache.flink - flink-scala{{ site.scala_version_suffix }} - {{site.version }} - - - org.apache.flink - flink-clients{{ site.scala_version_suffix }} - {{site.version }} - -{% endhighlight %} - -**Important:** When working with the Scala API you must have one of these two imports: -{% highlight scala %} -import org.apache.flink.api.scala._ -{% endhighlight %} - -or - -{% highlight scala %} -import org.apache.flink.api.scala.createTypeInformation -{% endhighlight %} - -The reason is that Flink analyzes the types that are used in a program and generates serializers -and comparators for them. By having either of those imports you enable an implicit conversion -that creates the type information for Flink operations. - -If you would rather use SBT, see [here]({{ site.baseurl }}/quickstart/scala_api_quickstart.html#sbt). -
    -
    - -#### Scala Dependency Versions - -Because Scala 2.10 binary is not compatible with Scala 2.11 binary, we provide multiple artifacts -to support both Scala versions. - -Starting from the 0.10 line, we cross-build all Flink modules for both 2.10 and 2.11. If you want -to run your program on Flink with Scala 2.11, you need to add a `_2.11` suffix to the `artifactId` -values of the Flink modules in your dependencies section. - -If you are looking for building Flink with Scala 2.11, please check -[build guide]({{ site.baseurl }}/start/building.html#scala-versions). - -#### Hadoop Dependency Versions - -If you are using Flink together with Hadoop, the version of the dependency may vary depending on the -version of Hadoop (or more specifically, HDFS) that you want to use Flink with. Please refer to the -[downloads page](http://flink.apache.org/downloads.html) for a list of available versions, and instructions -on how to link with custom versions of Hadoop. - -In order to link against the latest SNAPSHOT versions of the code, please follow -[this guide](http://flink.apache.org/how-to-contribute.html#snapshots-nightly-builds). - -The *flink-clients* dependency is only necessary to invoke the Flink program locally (for example to -run it standalone for testing and debugging). If you intend to only export the program as a JAR -file and [run it on a cluster]({{ site.baseurl }}/dev/cluster_execution.html), you can skip that dependency. - -{% top %} - diff --git a/docs/redirects/linking_with_flink.md b/docs/redirects/linking_with_flink.md new file mode 100644 index 0000000000000..1289487c9ff22 --- /dev/null +++ b/docs/redirects/linking_with_flink.md @@ -0,0 +1,25 @@ +--- +title: "Linking with Flink" +layout: redirect +redirect: /start/dependencies.html +permalink: /dev/linking_with_flink.html +--- + + diff --git a/docs/redirects/linking_with_optional_modules.md b/docs/redirects/linking_with_optional_modules.md new file mode 100644 index 0000000000000..e494fbc54d082 --- /dev/null +++ b/docs/redirects/linking_with_optional_modules.md @@ -0,0 +1,25 @@ +--- +title: "Linking with Optional Modules" +layout: redirect +redirect: /start/dependencies.html +permalink: /dev/linking.html +--- + + diff --git a/docs/start/dependencies.md b/docs/start/dependencies.md new file mode 100644 index 0000000000000..1375c6f30f8ea --- /dev/null +++ b/docs/start/dependencies.md @@ -0,0 +1,244 @@ +--- +title: "Configuring Dependencies, Connectors, Libraries" +nav-parent_id: start +nav-pos: 2 +--- + + +Every Flink application depends on a set of Flink libraries. At the bare minimum, the application depends +on the Flink APIs. Many applications depend in addition on certain connector libraries (like Kafka, Cassandra, etc.). +When running Flink applications (either in a distributed deployment, or in the IDE for testing), the Flink +runtime library must be available as well. + + +## Flink Core and Application Dependencies + +As with most systems that run user-defined applications, there are two broad categories of dependencies and libraries in Flink: + + - **Flink Core Dependenies**: Flink itself consists of a set of classes and dependencies that are needed to run the system, for example + coordination, networking, checkpoints, failover, APIs, operations (such as windowing), resource management, etc. + The set of all these classes and dependencies forms the core of Flink's runtime and must be present when a Flink + application is started. + + These core classes and dependencies are packaged in the `flink-dist` jar. They are part of Flink's `lib` folder and + part of the basic Flink container images. Think of these dependencies as similar to Java's core library (`rt.jar`, `charsets.jar`, etc.), + which contains the classes like `String` and `List`. + + The Flink Core Dependencies do not contain any connectors or libraries (CEP, SQL, ML, etc.) in order to avoid having an excessive + number of dependencies and classes in the classpath by default. In fact, we try to keep the core dependencies as slim as possible + to keep the default classpath small and avoid dependency clashes. + + - The **User Application Dependencies** are all connectors, formats, or libraries that a specific user application needs. + + The user application is typically packaged into an *application jar*, which contains the application code and the required + connector and library dependencies. + + The user application dependencies explicitly do not include the Flink DataSet / DataStream APIs and runtime dependencies, + because those are already part of Flink's Core Dependencies. + + +## Setting up a Project: Basic Dependencies + +Every Flink application needs as the bare minimum the API dependencies, to develop against. +For Maven, you can use the [Java Project Template]({{ site.baseurl }}/quickstart/java_api_quickstart.html) +or [Scala Project Template]({{ site.baseurl }}/quickstart/scala_api_quickstart.html) to create +a program skeleton with these initial dependencies. + +When setting up a project manually, you need to add the following dependencies for the Java/Scala API +(here presented in Maven syntax, but the same dependencies apply to other build tools (Gradle, SBT, etc.) as well. + +
    +
    +{% highlight xml %} + + org.apache.flink + flink-java + {{site.version }} + provided + + + org.apache.flink + flink-streaming-java{{ site.scala_version_suffix }} + {{site.version }} + provided + +{% endhighlight %} +
    +
    +{% highlight xml %} + + org.apache.flink + flink-scala{{ site.scala_version_suffix }} + {{site.version }} + provided + + + org.apache.flink + flink-streaming-scala{{ site.scala_version_suffix }} + {{site.version }} + provided + +{% endhighlight %} +
    +
    + +**Important:** Please note that all these dependencies have their scope set to *provided*. +That means that they are needed to compile against, but that they should not be packaged into the +project's resulting application jar file - these dependencies are Flink Core Dependencies, +which are already available in any setup. + +It is highly recommended to keep the dependencies in scope *provided*. If they are not set to *provided*, +the best case is that the resulting JAR becomes excessively large, because it also contains all Flink core +dependencies. The worst case is that the Flink core dependencies that are added to the application's jar file +clash with some of your own dependency versions (which is normally avoided through inverted classloading). + +**Note on IntelliJ:** To make the applications run within IntelliJ IDEA, the Flink dependencies need +to be declared in scope *compile* rather than *provided*. Otherwise IntelliJ will not add them to the classpath and +the in-IDE execution will fail with a `NoClassDefFountError`. To avoid having to declare the +dependency scope as *compile* (which is not recommended, see above), the above linked Java- and Scala +project templates use a trick: They add a profile that selectively activates when the application +is run in IntelliJ and only then promotes the dependencies to scope *compile*, without affecting +the packaging of the JAR files. + + +## Adding Connector and Library Dependencies + +Most applications need specific connectors or libraries to run, for example a connector to Kafka, Cassandra, etc. +These connectors are not part of Flink's core dependencies and must hence be added as dependencies to the application + +Below is an example adding the connector for Kafka 0.10 as a dependency (Maven syntax): +{% highlight xml %} + + org.apache.flink + flink-connector-kafka-0.10{{ site.scala_version_suffix }} + {{site.version }} + +{% endhighlight %} + +We recommend to package the application code and all its required dependencies into one *jar-with-dependencies* which +we refer to as the *application jar*. The application jar can be submitted to an already running Flink cluster, +or added to a Flink application container image. + +Projects created from the [Java Project Template]({{ site.baseurl }}/quickstart/java_api_quickstart.html) or +[Scala Project Template]({{ site.baseurl }}/quickstart/scala_api_quickstart.html) are configured to automatically include +the application dependencies into the application jar when running `mvn clean package`. For projects that are +not set up from those templates, we recommend to add the Maven Shade Plugin (as listed in the Appendix below) +to build the application jar with all required dependencies. + +**Important:** For Maven (and other build tools) to correctly package the dependencies into the application jar, +these application dependencies must be specified in scope *compile* (unlike the core dependencies, which +must be specified in scope *provided*). + + +## Scala Versions + +Scala versions (2.10, 2.11, 2.12, etc.) are not binary compatible with one another. +For that reason, Flink for Scala 2.11 cannot be used with an application that uses +Scala 2.12. + +All Flink dependencies that (transitively) depend on Scala are suffixed with the +Scala version that they are built for, for example `flink-streaming-scala_2.11`. + +Developers that only use Java can pick any Scala version, Scala developers need to +pick the Scala version that matches their application's Scala version. + +Please refer to the [build guide]({{ site.baseurl }}/start/building.html#scala-versions) +for details on how to build Flink for a specific Scala version. + +**Note:** Because of major breaking changes in Scala 2.12, Flink 1.5 currently builds only for Scala 2.11. +We aim to add support for Scala 2.12 in the next versions. + + +## Hadoop Dependencies + +**General rule: It should never be necessary to add Hadoop dependencies directly to your application.** +*(The only exception being when using existing Hadoop input-/output formats with Flink's Hadoop compatibility wrappers)* + +If you want to use Flink with Hadoop, you need to have a Flink setup that includes the Hadoop dependencies, rather than +adding Hadoop as an application dependency. Please refer to the [Hadoop Setup Guide]({{ site.baseurl }}/ops/deployment/hadoop.html) +for details. + +There are two main reasons for that design: + + - Some Hadoop interaction happens in Flink's core, possibly before the user application is started, for example + setting up HDFS for checkpoints, authenticating via Hadoop's Kerberos tokens, or deployment on YARN. + + - Flink's inverted classloading approach hides many transitive dependencies from the core dependencies. That applies not only + to Flink's own core dependencies, but also to Hadoop's dependencies when present in the setup. + That way, applications can use different versions of the same dependencies without running into dependency conflicts (and + trust us, that's a big deal, because Hadoops dependency tree is huge.) + +If you need Hadoop dependencies during testing or development inside the IDE (for example for HDFS access), please configure +these dependencies similar to the scope of the dependencies to *test* or to *provided*. + + +## Appendix: Template for bulding a Jar with Dependencies + +To build an application JAR that contains all dependencies required for declared connectors and libraries, +you can use the following shade plugin definition: + +{% highlight xml %} + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + package + + shade + + + + + com.google.code.findbugs:jsr305 + org.slf4j:* + log4j:* + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + my.prorgams.main.clazz + + + + + + + + +{% endhighlight %} + +{% top %} + From 2f6cb37c775106bb684ef9c608585e7a72056460 Mon Sep 17 00:00:00 2001 From: gyao Date: Tue, 27 Feb 2018 16:58:53 +0100 Subject: [PATCH 007/268] [FLINK-8787][flip6] Do not copy flinkConfiguration in AbstractYarnClusterDescriptor This closes #5591. --- .../org/apache/flink/yarn/AbstractYarnClusterDescriptor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index e6c36f6a2d29a..6b930163896c0 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -482,7 +482,7 @@ protected ClusterClient deployInternal( flinkConfiguration.setString(ClusterEntrypoint.EXECUTION_MODE, executionMode.toString()); ApplicationReport report = startAppMaster( - new Configuration(flinkConfiguration), + flinkConfiguration, yarnClusterEntrypoint, jobGraph, yarnClient, From acf114793c708f0ab207008c25195f6f65796e5f Mon Sep 17 00:00:00 2001 From: gyao Date: Wed, 21 Feb 2018 16:02:01 +0100 Subject: [PATCH 008/268] [FLINK-8730][REST] JSON serialize entire SerializedThrowable Do not only serialize the serialized exception but the entire SerializedThrowable object. This makes it possible to throw the SerializedThrowable itself without deserializing it. This closes #5546. --- .../flink/util/SerializedThrowable.java | 14 ---- .../json/SerializedThrowableDeserializer.java | 15 ++-- .../json/SerializedThrowableSerializer.java | 7 +- .../SerializedThrowableSerializerTest.java | 71 +++++++++++++++++++ 4 files changed, 83 insertions(+), 24 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableSerializerTest.java diff --git a/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java b/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java index de6358c7ce515..13f8d77d973a6 100644 --- a/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java +++ b/flink-core/src/main/java/org/apache/flink/util/SerializedThrowable.java @@ -25,8 +25,6 @@ import java.util.HashSet; import java.util.Set; -import static java.util.Objects.requireNonNull; - /** * Utility class for dealing with user-defined Throwable types that are serialized (for * example during RPC/Actor communication), but cannot be resolved with the default @@ -64,18 +62,6 @@ public SerializedThrowable(Throwable exception) { this(exception, new HashSet<>()); } - /** - * Creates a new SerializedThrowable from a serialized exception provided as a byte array. - */ - public SerializedThrowable( - final byte[] serializedException, - final String originalErrorClassName, - final String fullStringifiedStackTrace) { - this.serializedException = requireNonNull(serializedException); - this.originalErrorClassName = requireNonNull(originalErrorClassName); - this.fullStringifiedStackTrace = requireNonNull(fullStringifiedStackTrace); - } - private SerializedThrowable(Throwable exception, Set alreadySeen) { super(getMessageOrError(exception)); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableDeserializer.java index 3217cce8dad08..d0f71ce93d614 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableDeserializer.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rest.messages.json; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; @@ -27,9 +28,7 @@ import java.io.IOException; -import static org.apache.flink.runtime.rest.messages.json.SerializedThrowableSerializer.FIELD_NAME_CLASS; -import static org.apache.flink.runtime.rest.messages.json.SerializedThrowableSerializer.FIELD_NAME_SERIALIZED_EXCEPTION; -import static org.apache.flink.runtime.rest.messages.json.SerializedThrowableSerializer.FIELD_NAME_STACK_TRACE; +import static org.apache.flink.runtime.rest.messages.json.SerializedThrowableSerializer.FIELD_NAME_SERIALIZED_THROWABLE; /** * JSON deserializer for {@link SerializedThrowable}. @@ -48,10 +47,12 @@ public SerializedThrowable deserialize( final DeserializationContext ctxt) throws IOException { final JsonNode root = p.readValueAsTree(); - final String exceptionClassName = root.get(FIELD_NAME_CLASS).asText(); - final String stackTrace = root.get(FIELD_NAME_STACK_TRACE).asText(); - final byte[] serializedException = root.get(FIELD_NAME_SERIALIZED_EXCEPTION).binaryValue(); - return new SerializedThrowable(serializedException, exceptionClassName, stackTrace); + final byte[] serializedException = root.get(FIELD_NAME_SERIALIZED_THROWABLE).binaryValue(); + try { + return InstantiationUtil.deserializeObject(serializedException, ClassLoader.getSystemClassLoader()); + } catch (ClassNotFoundException e) { + throw new IOException("Failed to deserialize " + SerializedThrowable.class.getCanonicalName(), e); + } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableSerializer.java index cb921a9f5cb40..51f111f362495 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedThrowableSerializer.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.rest.messages.json; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; @@ -33,12 +34,12 @@ public class SerializedThrowableSerializer extends StdSerializer Date: Tue, 27 Feb 2018 14:43:52 +0800 Subject: [PATCH 009/268] [FLINK-8792] [rest] Change MessageQueryParameter.convertStringToValue to convertValueToString This closes #5587. --- .../handlers/AllowNonRestoredStateQueryParameter.java | 2 +- .../webmonitor/handlers/ParallelismQueryParameter.java | 2 +- .../runtime/webmonitor/handlers/StringQueryParameter.java | 2 +- .../handlers/AllowNonRestoredStateQueryParameterTest.java | 4 ++-- .../webmonitor/handlers/ParallelismQueryParameterTest.java | 2 +- .../flink/runtime/rest/messages/MessageQueryParameter.java | 6 +++--- .../rest/messages/RescalingParallelismQueryParameter.java | 2 +- .../rest/messages/TerminationModeQueryParameter.java | 2 +- .../rest/messages/job/metrics/MetricsFilterParameter.java | 2 +- .../apache/flink/runtime/rest/RestServerEndpointITCase.java | 2 +- .../runtime/rest/handler/util/HandlerRequestUtilsTest.java | 2 +- .../flink/runtime/rest/messages/MessageParametersTest.java | 2 +- .../messages/job/metrics/MetricsFilterParameterTest.java | 2 +- 13 files changed, 16 insertions(+), 16 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java index 7ad014ea685f8..2ddde3ae2fefe 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java @@ -39,7 +39,7 @@ public Boolean convertValueFromString(final String value) { } @Override - public String convertStringToValue(final Boolean value) { + public String convertValueToString(final Boolean value) { return value.toString(); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java index 26cb16c7b0e33..2ade7eb0a5cfb 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java @@ -38,7 +38,7 @@ public Integer convertValueFromString(final String value) { } @Override - public String convertStringToValue(final Integer value) { + public String convertValueToString(final Integer value) { return value.toString(); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java index 226c5929c5693..52c0967c7251e 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java @@ -35,7 +35,7 @@ public final String convertValueFromString(final String value) { } @Override - public final String convertStringToValue(final String value) { + public final String convertValueToString(final String value) { return value; } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java index 9882637e52137..97b61f635c89e 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java @@ -34,8 +34,8 @@ public class AllowNonRestoredStateQueryParameterTest extends TestLogger { @Test public void testConvertStringToValue() { - assertEquals("false", allowNonRestoredStateQueryParameter.convertStringToValue(false)); - assertEquals("true", allowNonRestoredStateQueryParameter.convertStringToValue(true)); + assertEquals("false", allowNonRestoredStateQueryParameter.convertValueToString(false)); + assertEquals("true", allowNonRestoredStateQueryParameter.convertValueToString(true)); } @Test diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java index 8189dd5bb93d6..684af219811d7 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java @@ -33,7 +33,7 @@ public class ParallelismQueryParameterTest extends TestLogger { @Test public void testConvertStringToValue() { - assertEquals("42", parallelismQueryParameter.convertStringToValue(42)); + assertEquals("42", parallelismQueryParameter.convertValueToString(42)); } @Test diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageQueryParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageQueryParameter.java index 506a14b8da505..29bee66d0cae2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageQueryParameter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageQueryParameter.java @@ -58,11 +58,11 @@ public String convertToString(List values) { boolean first = true; for (X value : values) { if (first) { - sb.append(convertStringToValue(value)); + sb.append(convertValueToString(value)); first = false; } else { sb.append(","); - sb.append(convertStringToValue(value)); + sb.append(convertValueToString(value)); } } return sb.toString(); @@ -74,5 +74,5 @@ public String convertToString(List values) { * @param value parameter value * @return string representation of typed value */ - public abstract String convertStringToValue(X value); + public abstract String convertValueToString(X value); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RescalingParallelismQueryParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RescalingParallelismQueryParameter.java index 9230d790a7b9e..be9eff1eec501 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RescalingParallelismQueryParameter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RescalingParallelismQueryParameter.java @@ -35,7 +35,7 @@ public Integer convertValueFromString(String value) { } @Override - public String convertStringToValue(Integer value) { + public String convertValueToString(Integer value) { return value.toString(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java index 9873f81ecd41e..386f22e225ce1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java @@ -37,7 +37,7 @@ public TerminationMode convertValueFromString(String value) { } @Override - public String convertStringToValue(TerminationMode value) { + public String convertValueToString(TerminationMode value) { return value.name().toLowerCase(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameter.java index b01d2a932dcaa..bcace79801f9b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameter.java @@ -41,7 +41,7 @@ public String convertValueFromString(String value) { } @Override - public String convertStringToValue(String value) { + public String convertValueToString(String value) { return value; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java index 3ad7ee50ecfae..e049a2d1e87ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java @@ -481,7 +481,7 @@ public JobID convertValueFromString(String value) { } @Override - public String convertStringToValue(JobID value) { + public String convertValueToString(JobID value) { return value.toString(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java index 09433f53021ca..259ae2040871f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java @@ -108,7 +108,7 @@ public Boolean convertValueFromString(final String value) { } @Override - public String convertStringToValue(final Boolean value) { + public String convertValueToString(final Boolean value) { return value.toString(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java index 65a1baa723541..03fcb0aa99b92 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java @@ -109,7 +109,7 @@ public JobID convertValueFromString(String value) { } @Override - public String convertStringToValue(JobID value) { + public String convertValueToString(JobID value) { return value.toString(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java index 7e1812f2e11cc..10dc921c3e76b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java @@ -49,7 +49,7 @@ public void testIsOptionalParameter() { @Test public void testConversions() { - assertThat(metricsFilterParameter.convertStringToValue("test"), equalTo("test")); + assertThat(metricsFilterParameter.convertValueToString("test"), equalTo("test")); assertThat(metricsFilterParameter.convertValueFromString("test"), equalTo("test")); } From 08e615027acd426537dc580139a61bd4082b7c3f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Feb 2018 10:11:44 +0100 Subject: [PATCH 010/268] [FLINK-8792] [rest] Change MessageQueryParameter#convertValueFromString to convertStringToValue --- .../handlers/AllowNonRestoredStateQueryParameter.java | 2 +- .../webmonitor/handlers/ParallelismQueryParameter.java | 2 +- .../runtime/webmonitor/handlers/StringQueryParameter.java | 2 +- .../handlers/AllowNonRestoredStateQueryParameterTest.java | 6 +++--- .../webmonitor/handlers/ParallelismQueryParameterTest.java | 2 +- .../flink/runtime/rest/messages/MessageQueryParameter.java | 4 ++-- .../rest/messages/RescalingParallelismQueryParameter.java | 2 +- .../rest/messages/TerminationModeQueryParameter.java | 2 +- .../rest/messages/job/metrics/MetricsFilterParameter.java | 2 +- .../apache/flink/runtime/rest/RestServerEndpointITCase.java | 2 +- .../runtime/rest/handler/util/HandlerRequestUtilsTest.java | 2 +- .../flink/runtime/rest/messages/MessageParametersTest.java | 2 +- .../messages/job/metrics/MetricsFilterParameterTest.java | 2 +- 13 files changed, 16 insertions(+), 16 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java index 2ddde3ae2fefe..19734d862b121 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameter.java @@ -34,7 +34,7 @@ protected AllowNonRestoredStateQueryParameter() { } @Override - public Boolean convertValueFromString(final String value) { + public Boolean convertStringToValue(final String value) { return Boolean.valueOf(value); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java index 2ade7eb0a5cfb..398bcb0647792 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameter.java @@ -33,7 +33,7 @@ public ParallelismQueryParameter() { } @Override - public Integer convertValueFromString(final String value) { + public Integer convertStringToValue(final String value) { return Integer.valueOf(value); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java index 52c0967c7251e..67e83ff8fe076 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/StringQueryParameter.java @@ -30,7 +30,7 @@ public StringQueryParameter(final String key, final MessageParameterRequisitenes } @Override - public final String convertValueFromString(final String value) { + public final String convertStringToValue(final String value) { return value; } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java index 97b61f635c89e..8bc1327d348b1 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/AllowNonRestoredStateQueryParameterTest.java @@ -40,9 +40,9 @@ public void testConvertStringToValue() { @Test public void testConvertValueFromString() { - assertEquals(false, allowNonRestoredStateQueryParameter.convertValueFromString("false")); - assertEquals(true, allowNonRestoredStateQueryParameter.convertValueFromString("true")); - assertEquals(true, allowNonRestoredStateQueryParameter.convertValueFromString("TRUE")); + assertEquals(false, allowNonRestoredStateQueryParameter.convertStringToValue("false")); + assertEquals(true, allowNonRestoredStateQueryParameter.convertStringToValue("true")); + assertEquals(true, allowNonRestoredStateQueryParameter.convertStringToValue("TRUE")); } } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java index 684af219811d7..cd9080bb9b2f0 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ParallelismQueryParameterTest.java @@ -38,7 +38,7 @@ public void testConvertStringToValue() { @Test public void testConvertValueFromString() { - assertEquals(42, (int) parallelismQueryParameter.convertValueFromString("42")); + assertEquals(42, (int) parallelismQueryParameter.convertStringToValue("42")); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageQueryParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageQueryParameter.java index 29bee66d0cae2..180f0119d98c9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageQueryParameter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/MessageQueryParameter.java @@ -39,7 +39,7 @@ public List convertFromString(String values) { String[] splitValues = values.split(","); List list = new ArrayList<>(); for (String value : splitValues) { - list.add(convertValueFromString(value)); + list.add(convertStringToValue(value)); } return list; } @@ -50,7 +50,7 @@ public List convertFromString(String values) { * @param value string representation of parameter value * @return parameter value */ - public abstract X convertValueFromString(String value); + public abstract X convertStringToValue(String value); @Override public String convertToString(List values) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RescalingParallelismQueryParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RescalingParallelismQueryParameter.java index be9eff1eec501..5c4f912a2d786 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RescalingParallelismQueryParameter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RescalingParallelismQueryParameter.java @@ -30,7 +30,7 @@ public RescalingParallelismQueryParameter() { } @Override - public Integer convertValueFromString(String value) { + public Integer convertStringToValue(String value) { return Integer.valueOf(value); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java index 386f22e225ce1..889b6d5621ec3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/TerminationModeQueryParameter.java @@ -32,7 +32,7 @@ public TerminationModeQueryParameter() { } @Override - public TerminationMode convertValueFromString(String value) { + public TerminationMode convertStringToValue(String value) { return TerminationMode.valueOf(value.toUpperCase()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameter.java index bcace79801f9b..9c6c0fd66b41a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameter.java @@ -36,7 +36,7 @@ public MetricsFilterParameter() { } @Override - public String convertValueFromString(String value) { + public String convertStringToValue(String value) { return value; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java index e049a2d1e87ea..c9817ff19e437 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java @@ -476,7 +476,7 @@ static class JobIDQueryParameter extends MessageQueryParameter { } @Override - public JobID convertValueFromString(String value) { + public JobID convertStringToValue(String value) { return JobID.fromHexString(value); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java index 259ae2040871f..5001e3daf6a31 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/util/HandlerRequestUtilsTest.java @@ -103,7 +103,7 @@ private TestBooleanQueryParameter() { } @Override - public Boolean convertValueFromString(final String value) { + public Boolean convertStringToValue(final String value) { return Boolean.parseBoolean(value); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java index 03fcb0aa99b92..8d73231350a09 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java @@ -104,7 +104,7 @@ private static class TestQueryParameter extends MessageQueryParameter { } @Override - public JobID convertValueFromString(String value) { + public JobID convertStringToValue(String value) { return JobID.fromHexString(value); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java index 10dc921c3e76b..b13cb01f64700 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java @@ -50,7 +50,7 @@ public void testIsOptionalParameter() { @Test public void testConversions() { assertThat(metricsFilterParameter.convertValueToString("test"), equalTo("test")); - assertThat(metricsFilterParameter.convertValueFromString("test"), equalTo("test")); + assertThat(metricsFilterParameter.convertStringToValue("test"), equalTo("test")); } } From 302aaeb021bacf3f37cb9a3ee236304c94adbf30 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 22 Feb 2018 17:22:54 +0100 Subject: [PATCH 011/268] [FLINK-8451] [serializers] Make Scala tuple serializer deserialization more failure tolerant This closes #5567. --- .../TupleSerializerConfigSnapshot.java | 2 +- .../apache/flink/util/InstantiationUtil.java | 56 +++++++++-- .../flink-1.3.2-scala-types-serializer-data | Bin 0 -> 97 bytes ...link-1.3.2-scala-types-serializer-snapshot | Bin 0 -> 7634 bytes .../TupleSerializerCompatibilityTest.scala | 86 ++++++++++++++++ ...SerializerCompatibilityTestGenerator.scala | 94 ++++++++++++++++++ 6 files changed, 231 insertions(+), 7 deletions(-) create mode 100644 flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-data create mode 100644 flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-snapshot create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerCompatibilityTest.scala create mode 100644 flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerCompatibilityTestGenerator.scala diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java index 705099e9b2dbc..eac5200da9c6f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java @@ -61,7 +61,7 @@ public void read(DataInputView in) throws IOException { super.read(in); try (final DataInputViewStream inViewWrapper = new DataInputViewStream(in)) { - tupleClass = InstantiationUtil.deserializeObject(inViewWrapper, getUserCodeClassLoader()); + tupleClass = InstantiationUtil.deserializeObject(inViewWrapper, getUserCodeClassLoader(), true); } catch (ClassNotFoundException e) { throw new IOException("Could not find requested tuple class in classpath.", e); } diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java index 11e3990077a59..978d270e05f97 100644 --- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java @@ -113,7 +113,7 @@ protected Class resolveClass(ObjectStreamClass desc) throws IOException, Clas * *

    This can be removed once 1.2 is no longer supported. */ - private static Set scalaSerializerClassnames = new HashSet<>(); + private static final Set scalaSerializerClassnames = new HashSet<>(); static { scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.TraversableSerializer"); scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.CaseClassSerializer"); @@ -121,10 +121,53 @@ protected Class resolveClass(ObjectStreamClass desc) throws IOException, Clas scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.EnumValueSerializer"); scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.OptionSerializer"); scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.TrySerializer"); - scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.EitherSerializer"); scalaSerializerClassnames.add("org.apache.flink.api.scala.typeutils.UnitSerializer"); } + /** + * The serialVersionUID might change between Scala versions and since those classes are + * part of the tuple serializer config snapshots we need to ignore them. + * + * @see FLINK-8451 + */ + private static final Set scalaTypes = new HashSet<>(); + static { + scalaTypes.add("scala.Tuple1"); + scalaTypes.add("scala.Tuple2"); + scalaTypes.add("scala.Tuple3"); + scalaTypes.add("scala.Tuple4"); + scalaTypes.add("scala.Tuple5"); + scalaTypes.add("scala.Tuple6"); + scalaTypes.add("scala.Tuple7"); + scalaTypes.add("scala.Tuple8"); + scalaTypes.add("scala.Tuple9"); + scalaTypes.add("scala.Tuple10"); + scalaTypes.add("scala.Tuple11"); + scalaTypes.add("scala.Tuple12"); + scalaTypes.add("scala.Tuple13"); + scalaTypes.add("scala.Tuple14"); + scalaTypes.add("scala.Tuple15"); + scalaTypes.add("scala.Tuple16"); + scalaTypes.add("scala.Tuple17"); + scalaTypes.add("scala.Tuple18"); + scalaTypes.add("scala.Tuple19"); + scalaTypes.add("scala.Tuple20"); + scalaTypes.add("scala.Tuple21"); + scalaTypes.add("scala.Tuple22"); + scalaTypes.add("scala.Tuple1$mcJ$sp"); + scalaTypes.add("scala.Tuple1$mcI$sp"); + scalaTypes.add("scala.Tuple1$mcD$sp"); + scalaTypes.add("scala.Tuple2$mcJJ$sp"); + scalaTypes.add("scala.Tuple2$mcJI$sp"); + scalaTypes.add("scala.Tuple2$mcJD$sp"); + scalaTypes.add("scala.Tuple2$mcIJ$sp"); + scalaTypes.add("scala.Tuple2$mcII$sp"); + scalaTypes.add("scala.Tuple2$mcID$sp"); + scalaTypes.add("scala.Tuple2$mcDJ$sp"); + scalaTypes.add("scala.Tuple2$mcDI$sp"); + scalaTypes.add("scala.Tuple2$mcDD$sp"); + } + /** * An {@link ObjectInputStream} that ignores serialVersionUID mismatches when deserializing objects of * anonymous classes or our Scala serializer classes and also replaces occurences of GenericData.Array @@ -158,12 +201,13 @@ protected ObjectStreamClass readClassDescriptor() throws IOException, ClassNotFo } } - Class localClass = resolveClass(streamClassDescriptor); - if (scalaSerializerClassnames.contains(localClass.getName()) || localClass.isAnonymousClass() + final Class localClass = resolveClass(streamClassDescriptor); + final String name = localClass.getName(); + if (scalaSerializerClassnames.contains(name) || scalaTypes.contains(name) || localClass.isAnonymousClass() // isAnonymousClass does not work for anonymous Scala classes; additionally check by classname - || localClass.getName().contains("$anon$") || localClass.getName().contains("$anonfun")) { + || name.contains("$anon$") || name.contains("$anonfun")) { - ObjectStreamClass localClassDescriptor = ObjectStreamClass.lookup(localClass); + final ObjectStreamClass localClassDescriptor = ObjectStreamClass.lookup(localClass); if (localClassDescriptor != null && localClassDescriptor.getSerialVersionUID() != streamClassDescriptor.getSerialVersionUID()) { LOG.warn("Ignoring serialVersionUID mismatch for anonymous class {}; was {}, now {}.", diff --git a/flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-data b/flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-data new file mode 100644 index 0000000000000000000000000000000000000000..ddd6ac01112405f127b5c3f3d9412bf060009d3f GIT binary patch literal 97 zcmZQzV9;V@^GMCf$!B0I&qyq>chE=!0|o{L9w^oV(ioB{K*1A0oC4L(2vwm2(f%I@ H8i3LO{3sa1 literal 0 HcmV?d00001 diff --git a/flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-snapshot b/flink-scala/src/test/resources/flink-1.3.2-scala-types-serializer-snapshot new file mode 100644 index 0000000000000000000000000000000000000000..1ebdabb29fc94d31123061db091237561394f4f2 GIT binary patch literal 7634 zcmeHMO>7%Q6n?v@LzR-Gv=BiB5>!6)(p|-A%8!vM#r=_sC6FA1+5*+Yo!FDEcg^g$ z#1%i{0yo5=2jGejrQ*f`Rq6#eASzB=xKyf&%AZ8V1#tn9zM0*4H_60}z3a*dTUy(* z*1mb~dw<_N1AqbQHTosMBVXZHT#sJ<>=tBv4ljDVYLN!1)Tp)SF!wAjnB`Z9LoD8O z1*=nQzS(eSnQ}%PcAoNrS8ou(7MR0CYo7Yz7^P3_m!lMwnN;it#0KJlLJ~~0?9FR)=G{nfaL}*GAk=9|2BE3=tgheL0taH zw;ulF^3>n&OMI0)j0?T9Hb`1PK&_%HBD3SlahThheM-z)Bg@RKI#hUW>(h5GePg|P zL~8d6yrH6YRx37$+O|KeBTUpNk5qf}PlJA#eS6jfvjh_kU5u$M;9##_p5&}ri;0p- zj<;3KYgN?)Sr#0vjueh#DW#5~V`jAS)|>vk%VHWm^Qe`*c5&v%RS2b=Gack?l0Aj3 zCjM+#0H%Nb=lCxz^TgMy8_P;$s%Vww+#^@0%F|j^^bpN?o>^%L=f zA>z5EeU+)!d*8m>0dZ;Jj`rIggzBW$0lhW>*%I1Se4qKn$A=z#DsMuj2nIV0rkE~) zK?UqCsSRvd_U#~5r3XY#z(n{{;5)p#P2?DpD(_VV%7CI)YS|?Xzvc<_ zl#I$u=&|2$iT6gz*dspz7(*P#F9J+l0GQkhuqy}fz}4`X8(|L}%vzaJb1~qU7V8b^ z`5fV-P6g!-_5HQAJV0EbmIN#Glkr1{`M`GoBbDynWf|(!K>E^*W%*8{BgynL;J7x@ zHp%Ms-Cdj3n7-=uJBmM#-R5PyZmPWiZib=MO`X5101svcCxlBD+iM~6Nh2rqkRD9L zdx0s4@sSror2EM|v}|v)6U2I=+9nDSqZ=Og)^<--z_5NvN+*wJ0fA2MxR)vgPdz0i z>yv|ugk?v9s1TI$#*!TmCi_J~SG(0M)b%xVbPcb+q~dt}6lQX2ho#g#HJHUSDP9u$ zAybf(uGlS?K{hkV Date: Mon, 26 Feb 2018 03:54:53 +0100 Subject: [PATCH 012/268] [FLINK-8777][checkpointing] Cleanup local state more eagerly in recovery (cherry picked from commit 296f9ff) --- .../runtime/state/TaskLocalStateStore.java | 8 ++ .../state/TaskLocalStateStoreImpl.java | 108 +++++++++++++----- .../runtime/state/TaskStateManagerImpl.java | 6 +- .../state/TaskLocalStateStoreImplTest.java | 42 +++++-- .../state/TestTaskLocalStateStore.java | 6 + 5 files changed, 130 insertions(+), 40 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java index 7089894c1c5af..686f4f6936f0c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStore.java @@ -24,6 +24,8 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.function.LongPredicate; + /** * Classes that implement this interface serve as a task-manager-level local storage for local checkpointed state. * The purpose is to provide access to a state that is stored locally for a faster recovery compared to the state that @@ -62,4 +64,10 @@ void storeLocalState( * and removes all local states with a checkpoint id that is smaller than the newly confirmed checkpoint id. */ void confirmCheckpoint(long confirmedCheckpointId); + + /** + * Remove all checkpoints from the store that match the given predicate. + * @param matcher the predicate that selects the checkpoints for pruning. + */ + void pruneMatchingCheckpoints(LongPredicate matcher); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java index bb4f0116dffe5..29adc4ada399e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskLocalStateStoreImpl.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.state; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; @@ -35,9 +36,10 @@ import java.io.File; import java.io.IOException; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -46,6 +48,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; +import java.util.function.LongPredicate; /** * Main implementation of a {@link TaskLocalStateStore}. @@ -56,7 +59,8 @@ public class TaskLocalStateStoreImpl implements TaskLocalStateStore { private static final Logger LOG = LoggerFactory.getLogger(TaskLocalStateStoreImpl.class); /** Dummy value to use instead of null to satisfy {@link ConcurrentHashMap}. */ - private static final TaskStateSnapshot NULL_DUMMY = new TaskStateSnapshot(0); + @VisibleForTesting + static final TaskStateSnapshot NULL_DUMMY = new TaskStateSnapshot(0); /** JobID from the owning subtask. */ @Nonnull @@ -103,14 +107,36 @@ public TaskLocalStateStoreImpl( @Nonnull LocalRecoveryConfig localRecoveryConfig, @Nonnull Executor discardExecutor) { - this.lock = new Object(); - this.storedTaskStateByCheckpointID = new TreeMap<>(); + this( + jobID, + allocationID, + jobVertexID, + subtaskIndex, + localRecoveryConfig, + discardExecutor, + new TreeMap<>(), + new Object()); + } + + @VisibleForTesting + TaskLocalStateStoreImpl( + @Nonnull JobID jobID, + @Nonnull AllocationID allocationID, + @Nonnull JobVertexID jobVertexID, + @Nonnegative int subtaskIndex, + @Nonnull LocalRecoveryConfig localRecoveryConfig, + @Nonnull Executor discardExecutor, + @Nonnull SortedMap storedTaskStateByCheckpointID, + @Nonnull Object lock) { + this.jobID = jobID; this.allocationID = allocationID; this.jobVertexID = jobVertexID; this.subtaskIndex = subtaskIndex; this.discardExecutor = discardExecutor; this.localRecoveryConfig = localRecoveryConfig; + this.storedTaskStateByCheckpointID = storedTaskStateByCheckpointID; + this.lock = lock; this.disposed = false; } @@ -133,23 +159,25 @@ public void storeLocalState( checkpointId, jobID, jobVertexID, subtaskIndex); } - Map toDiscard = new HashMap<>(16); + Map.Entry toDiscard = null; synchronized (lock) { if (disposed) { // we ignore late stores and simply discard the state. - toDiscard.put(checkpointId, localState); + toDiscard = new AbstractMap.SimpleEntry<>(checkpointId, localState); } else { TaskStateSnapshot previous = storedTaskStateByCheckpointID.put(checkpointId, localState); if (previous != null) { - toDiscard.put(checkpointId, previous); + toDiscard = new AbstractMap.SimpleEntry<>(checkpointId, previous); } } } - asyncDiscardLocalStateForCollection(toDiscard.entrySet()); + if (toDiscard != null) { + asyncDiscardLocalStateForCollection(Collections.singletonList(toDiscard)); + } } @Override @@ -157,10 +185,13 @@ public void storeLocalState( public TaskStateSnapshot retrieveLocalState(long checkpointID) { TaskStateSnapshot snapshot; + synchronized (lock) { snapshot = storedTaskStateByCheckpointID.get(checkpointID); } + snapshot = (snapshot != NULL_DUMMY) ? snapshot : null; + if (LOG.isTraceEnabled()) { LOG.trace("Found entry for local state for checkpoint {} in subtask ({} - {} - {}) : {}", checkpointID, jobID, jobVertexID, subtaskIndex, snapshot); @@ -169,7 +200,7 @@ public TaskStateSnapshot retrieveLocalState(long checkpointID) { checkpointID, jobID, jobVertexID, subtaskIndex); } - return snapshot != NULL_DUMMY ? snapshot : null; + return snapshot; } @Override @@ -184,30 +215,18 @@ public void confirmCheckpoint(long confirmedCheckpointId) { LOG.debug("Received confirmation for checkpoint {} in subtask ({} - {} - {}). Starting to prune history.", confirmedCheckpointId, jobID, jobVertexID, subtaskIndex); - final List> toRemove = new ArrayList<>(); - - synchronized (lock) { - - Iterator> entryIterator = - storedTaskStateByCheckpointID.entrySet().iterator(); + pruneCheckpoints( + (snapshotCheckpointId) -> snapshotCheckpointId < confirmedCheckpointId, + true); - // remove entries for outdated checkpoints and discard their state. - while (entryIterator.hasNext()) { - - Map.Entry snapshotEntry = entryIterator.next(); - long entryCheckpointId = snapshotEntry.getKey(); + } - if (entryCheckpointId < confirmedCheckpointId) { - toRemove.add(snapshotEntry); - entryIterator.remove(); - } else { - // we can stop because the map is sorted. - break; - } - } - } + @Override + public void pruneMatchingCheckpoints(@Nonnull LongPredicate matcher) { - asyncDiscardLocalStateForCollection(toRemove); + pruneCheckpoints( + matcher, + false); } /** @@ -300,6 +319,35 @@ private void deleteDirectory(File directory) throws IOException { } } + /** + * Pruning the useless checkpoints, it should be called only when holding the {@link #lock}. + */ + private void pruneCheckpoints(LongPredicate pruningChecker, boolean breakOnceCheckerFalse) { + + final List> toRemove = new ArrayList<>(); + + synchronized (lock) { + + Iterator> entryIterator = + storedTaskStateByCheckpointID.entrySet().iterator(); + + while (entryIterator.hasNext()) { + + Map.Entry snapshotEntry = entryIterator.next(); + long entryCheckpointId = snapshotEntry.getKey(); + + if (pruningChecker.test(entryCheckpointId)) { + toRemove.add(snapshotEntry); + entryIterator.remove(); + } else if (breakOnceCheckerFalse) { + break; + } + } + } + + asyncDiscardLocalStateForCollection(toRemove); + } + @Override public String toString() { return "TaskLocalStateStore{" + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java index e057d1957440c..e542ba13fda3a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateManagerImpl.java @@ -119,8 +119,12 @@ public PrioritizedOperatorSubtaskState prioritizedOperatorState(OperatorID opera return PrioritizedOperatorSubtaskState.emptyNotRestored(); } + long restoreCheckpointId = jobManagerTaskRestore.getRestoreCheckpointId(); + TaskStateSnapshot localStateSnapshot = - localStateStore.retrieveLocalState(jobManagerTaskRestore.getRestoreCheckpointId()); + localStateStore.retrieveLocalState(restoreCheckpointId); + + localStateStore.pruneMatchingCheckpoints((long checkpointId) -> checkpointId != restoreCheckpointId); List alternativesByPriority = Collections.emptyList(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskLocalStateStoreImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskLocalStateStoreImplTest.java index 16416762a525b..618320ebc4f39 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskLocalStateStoreImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskLocalStateStoreImplTest.java @@ -36,28 +36,30 @@ import java.io.File; import java.util.ArrayList; import java.util.List; +import java.util.SortedMap; +import java.util.TreeMap; import static org.powermock.api.mockito.PowerMockito.spy; public class TaskLocalStateStoreImplTest { + private SortedMap internalSnapshotMap; + private Object internalLock; private TemporaryFolder temporaryFolder; - private JobID jobID; - private AllocationID allocationID; - private JobVertexID jobVertexID; - private int subtaskIdx; private File[] allocationBaseDirs; private TaskLocalStateStoreImpl taskLocalStateStore; @Before public void before() throws Exception { + JobID jobID = new JobID(); + AllocationID allocationID = new AllocationID(); + JobVertexID jobVertexID = new JobVertexID(); + int subtaskIdx = 0; this.temporaryFolder = new TemporaryFolder(); this.temporaryFolder.create(); - this.jobID = new JobID(); - this.allocationID = new AllocationID(); - this.jobVertexID = new JobVertexID(); - this.subtaskIdx = 0; this.allocationBaseDirs = new File[]{temporaryFolder.newFolder(), temporaryFolder.newFolder()}; + this.internalSnapshotMap = new TreeMap<>(); + this.internalLock = new Object(); LocalRecoveryDirectoryProviderImpl directoryProvider = new LocalRecoveryDirectoryProviderImpl(allocationBaseDirs, jobID, jobVertexID, subtaskIdx); @@ -71,7 +73,9 @@ public void before() throws Exception { jobVertexID, subtaskIdx, localRecoveryConfig, - Executors.directExecutor()); + Executors.directExecutor(), + internalSnapshotMap, + internalLock); } @After @@ -116,6 +120,26 @@ public void storeAndRetrieve() throws Exception { Assert.assertNull(taskLocalStateStore.retrieveLocalState(chkCount + 1)); } + /** + * Test checkpoint pruning. + */ + @Test + public void pruneCheckpoints() throws Exception { + + final int chkCount = 3; + + List taskStateSnapshots = storeStates(chkCount); + + // test retrieve with pruning + taskLocalStateStore.pruneMatchingCheckpoints((long chk) -> chk != chkCount - 1); + + for (int i = 0; i < chkCount - 1; ++i) { + Assert.assertNull(taskLocalStateStore.retrieveLocalState(i)); + } + + checkStoredAsExpected(taskStateSnapshots, chkCount - 1, chkCount); + } + /** * Tests pruning of previous checkpoints if a new checkpoint is confirmed. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskLocalStateStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskLocalStateStore.java index 12c07ddd59542..2ade3e6d14262 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskLocalStateStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TestTaskLocalStateStore.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.SortedMap; import java.util.TreeMap; +import java.util.function.LongPredicate; /** * Test implementation of a {@link TaskLocalStateStore}. @@ -103,6 +104,11 @@ public void confirmCheckpoint(long confirmedCheckpointId) { } } + @Override + public void pruneMatchingCheckpoints(LongPredicate matcher) { + taskStateSnapshotsByCheckpointID.keySet().removeIf(matcher::test); + } + public boolean isDisposed() { return disposed; } From cf854ccbc6fdbf112095c471705c8799aee64a45 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 28 Feb 2018 15:35:13 +0100 Subject: [PATCH 013/268] [hotfix] Enable FILESYTEM_DEFAULT_OVERRIDE in FLIP-6 MiniClusterResource --- .../java/org/apache/flink/test/util/MiniClusterResource.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java index 9bb1ae9ec7211..1c5da62b2e000 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.minicluster.JobExecutorService; @@ -144,6 +145,10 @@ private JobExecutorService startOldMiniCluster() throws Exception { private JobExecutorService startFlip6MiniCluster() throws Exception { final Configuration configuration = miniClusterResourceConfiguration.getConfiguration(); + // we need to set this since a lot of test expect this because TestBaseUtils.startCluster() + // enabled this by default + configuration.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); + // set rest port to 0 to avoid clashes with concurrent MiniClusters configuration.setInteger(RestOptions.REST_PORT, 0); From 623e94459795a191703b880fcfa4f162c92ae458 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Wed, 28 Feb 2018 14:25:55 +0100 Subject: [PATCH 014/268] [FLINK-8557][checkpointing] Remove illegal characters from operator description text before using it to construct the instance directory in RocksDB (cherry picked from commit 66474da) --- .../contrib/streaming/state/RocksDBStateBackend.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java index f60cb2cd82dec..93892952f88b6 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java @@ -396,10 +396,14 @@ public AbstractKeyedStateBackend createKeyedStateBackend( String tempDir = env.getTaskManagerInfo().getTmpDirectories()[0]; ensureRocksDBIsLoaded(tempDir); - lazyInitializeForJob(env, operatorIdentifier); + // replace all characters that are not legal for filenames with underscore + String fileCompatibleIdentifier = operatorIdentifier.replaceAll("[^a-zA-Z0-9\\-]", "_"); - File instanceBasePath = - new File(getNextStoragePath(), "job-" + jobId + "_op-" + operatorIdentifier + "_uuid-" + UUID.randomUUID()); + lazyInitializeForJob(env, fileCompatibleIdentifier); + + File instanceBasePath = new File( + getNextStoragePath(), + "job_" + jobId + "_op_" + fileCompatibleIdentifier + "_uuid_" + UUID.randomUUID()); LocalRecoveryConfig localRecoveryConfig = env.getTaskStateManager().createLocalRecoveryConfig(); From bb459cc68f8dc4bd042b61e365e583d4e96b3e0e Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 23 Feb 2018 11:37:37 +0100 Subject: [PATCH 015/268] [hotfix][tests] Deduplicate code in SingleInputGateTest (cherry picked from commit 67a547a) --- .../consumer/SingleInputGateTest.java | 66 +++++++------------ 1 file changed, 25 insertions(+), 41 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 0dd0875603213..e94411dac3d15 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -77,14 +77,7 @@ public class SingleInputGateTest { @Test(timeout = 120 * 1000) public void testBasicGetNextLogic() throws Exception { // Setup - final SingleInputGate inputGate = new SingleInputGate( - "Test Task Name", new JobID(), - new IntermediateDataSetID(), ResultPartitionType.PIPELINED, - 0, 2, - mock(TaskActions.class), - UnregisteredMetricGroups.createUnregisteredTaskMetricGroup().getIOMetricGroup()); - - assertEquals(ResultPartitionType.PIPELINED, inputGate.getConsumedPartitionType()); + final SingleInputGate inputGate = createInputGate(); final TestInputChannel[] inputChannels = new TestInputChannel[]{ new TestInputChannel(inputGate, 0), @@ -135,14 +128,8 @@ public void testBackwardsEventWithUninitializedChannel() throws Exception { any(BufferAvailabilityListener.class))).thenReturn(iterator); // Setup reader with one local and one unknown input channel - final IntermediateDataSetID resultId = new IntermediateDataSetID(); - final SingleInputGate inputGate = new SingleInputGate( - "Test Task Name", new JobID(), - resultId, ResultPartitionType.PIPELINED, - 0, 2, - mock(TaskActions.class), - UnregisteredMetricGroups.createUnregisteredTaskMetricGroup().getIOMetricGroup()); + final SingleInputGate inputGate = createInputGate(); final BufferPool bufferPool = mock(BufferPool.class); when(bufferPool.getNumberOfRequiredMemorySegments()).thenReturn(2); @@ -190,14 +177,7 @@ public void testBackwardsEventWithUninitializedChannel() throws Exception { */ @Test public void testUpdateChannelBeforeRequest() throws Exception { - SingleInputGate inputGate = new SingleInputGate( - "t1", - new JobID(), - new IntermediateDataSetID(), - ResultPartitionType.PIPELINED, - 0, - 1, - mock(TaskActions.class), UnregisteredMetricGroups.createUnregisteredTaskMetricGroup().getIOMetricGroup()); + SingleInputGate inputGate = createInputGate(1); ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); @@ -230,15 +210,7 @@ public void testReleaseWhilePollingChannel() throws Exception { final AtomicReference asyncException = new AtomicReference<>(); // Setup the input gate with a single channel that does nothing - final SingleInputGate inputGate = new SingleInputGate( - "InputGate", - new JobID(), - new IntermediateDataSetID(), - ResultPartitionType.PIPELINED, - 0, - 1, - mock(TaskActions.class), - UnregisteredMetricGroups.createUnregisteredTaskMetricGroup().getIOMetricGroup()); + final SingleInputGate inputGate = createInputGate(1); InputChannel unknown = new UnknownInputChannel( inputGate, @@ -410,15 +382,7 @@ public void testRequestBuffersWithRemoteInputChannel() throws Exception { */ @Test public void testRequestBuffersWithUnknownInputChannel() throws Exception { - final SingleInputGate inputGate = new SingleInputGate( - "t1", - new JobID(), - new IntermediateDataSetID(), - ResultPartitionType.PIPELINED_BOUNDED, - 0, - 1, - mock(TaskActions.class), - UnregisteredMetricGroups.createUnregisteredTaskMetricGroup().getIOMetricGroup()); + final SingleInputGate inputGate = createInputGate(1); UnknownInputChannel unknown = mock(UnknownInputChannel.class); final ResultPartitionID resultPartitionId = new ResultPartitionID(); @@ -443,6 +407,26 @@ public void testRequestBuffersWithUnknownInputChannel() throws Exception { // --------------------------------------------------------------------------------------------- + private static SingleInputGate createInputGate() { + return createInputGate(2); + } + + private static SingleInputGate createInputGate(int numberOfInputChannels) { + SingleInputGate inputGate = new SingleInputGate( + "Test Task Name", + new JobID(), + new IntermediateDataSetID(), + ResultPartitionType.PIPELINED, + 0, + numberOfInputChannels, + mock(TaskActions.class), + UnregisteredMetricGroups.createUnregisteredTaskMetricGroup().getIOMetricGroup()); + + assertEquals(ResultPartitionType.PIPELINED, inputGate.getConsumedPartitionType()); + + return inputGate; + } + static void verifyBufferOrEvent( InputGate inputGate, boolean isBuffer, From 6b7a4480ef8610df3ff21eb2811b9a0a3c58c912 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 23 Feb 2018 12:11:14 +0100 Subject: [PATCH 016/268] [hotfix][runtime] Remove duplicated check (cherry picked from commit 42f71f6) --- .../io/network/partition/consumer/UnionInputGate.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index 5a547ea53b246..481599ce36ec4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -177,13 +177,6 @@ public Optional getNextBufferOrEvent() throws IOException, Interr } } - if (bufferOrEvent.moreAvailable()) { - // this buffer or event was now removed from the non-empty gates queue - // we re-add it in case it has more data, because in that case no "non-empty" notification - // will come for that gate - queueInputGate(inputGate); - } - // Set the channel index to identify the input channel (across all unioned input gates) final int channelIndexOffset = inputGateToIndexOffsetMap.get(inputGate); From 651462e6b22c51ce14bd9ea6db389ef6a1f38e55 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 23 Feb 2018 11:20:21 +0100 Subject: [PATCH 017/268] [FLINK-8760][runtime] Correctly propagate moreAvailable flag through SingleInputGate Previously if we SingleInputGate was re-eqnqueuing an input channel, isMoreAvailable might incorrectly return false. This might caused some dead locks. (cherry picked from commit 6c9e267) --- .../partition/consumer/BufferOrEvent.java | 6 +- .../partition/consumer/SingleInputGate.java | 1 + .../partition/consumer/UnionInputGate.java | 9 ++- .../consumer/SingleInputGateTest.java | 62 +++++++++++++++---- .../partition/consumer/TestInputChannel.java | 14 ++++- .../consumer/UnionInputGateTest.java | 33 +++++----- 6 files changed, 92 insertions(+), 33 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java index 3e93ae6e6880c..d1da4388c1b2b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/BufferOrEvent.java @@ -39,7 +39,7 @@ public class BufferOrEvent { * This is not needed outside of the input gate unioning logic and cannot * be set outside of the consumer package. */ - private final boolean moreAvailable; + private boolean moreAvailable; private int channelIndex; @@ -99,4 +99,8 @@ public String toString() { return String.format("BufferOrEvent [%s, channelIndex = %d]", isBuffer() ? buffer : event, channelIndex); } + + public void setMoreAvailable(boolean moreAvailable) { + this.moreAvailable = moreAvailable; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index a1f3cdcc5b4b7..be4035c55b173 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -540,6 +540,7 @@ private Optional getNextBufferOrEvent(boolean blocking) throws IO // will come for that channel if (result.get().moreAvailable()) { queueChannel(currentChannel); + moreAvailable = true; } final Buffer buffer = result.get().buffer(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index 481599ce36ec4..393e08775141a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -181,6 +181,7 @@ public Optional getNextBufferOrEvent() throws IOException, Interr final int channelIndexOffset = inputGateToIndexOffsetMap.get(inputGate); bufferOrEvent.setChannelIndex(channelIndexOffset + bufferOrEvent.getChannelIndex()); + bufferOrEvent.setMoreAvailable(bufferOrEvent.moreAvailable() || inputGateWithData.moreInputGatesAvailable); return Optional.ofNullable(bufferOrEvent); } @@ -193,18 +194,20 @@ public Optional pollNextBufferOrEvent() throws IOException, Inter private InputGateWithData waitAndGetNextInputGate() throws IOException, InterruptedException { while (true) { InputGate inputGate; + boolean moreInputGatesAvailable; synchronized (inputGatesWithData) { while (inputGatesWithData.size() == 0) { inputGatesWithData.wait(); } inputGate = inputGatesWithData.remove(); enqueuedInputGatesWithData.remove(inputGate); + moreInputGatesAvailable = enqueuedInputGatesWithData.size() > 0; } // In case of inputGatesWithData being inaccurate do not block on an empty inputGate, but just poll the data. Optional bufferOrEvent = inputGate.pollNextBufferOrEvent(); if (bufferOrEvent.isPresent()) { - return new InputGateWithData(inputGate, bufferOrEvent.get()); + return new InputGateWithData(inputGate, bufferOrEvent.get(), moreInputGatesAvailable); } } } @@ -212,10 +215,12 @@ private InputGateWithData waitAndGetNextInputGate() throws IOException, Interrup private static class InputGateWithData { private final InputGate inputGate; private final BufferOrEvent bufferOrEvent; + private final boolean moreInputGatesAvailable; - public InputGateWithData(InputGate inputGate, BufferOrEvent bufferOrEvent) { + public InputGateWithData(InputGate inputGate, BufferOrEvent bufferOrEvent, boolean moreInputGatesAvailable) { this.inputGate = checkNotNull(inputGate); this.bufferOrEvent = checkNotNull(bufferOrEvent); + this.moreInputGatesAvailable = moreInputGatesAvailable; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index e94411dac3d15..8c54c1f6ad6a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -100,15 +100,40 @@ public void testBasicGetNextLogic() throws Exception { inputGate.notifyChannelNonEmpty(inputChannels[0].getInputChannel()); inputGate.notifyChannelNonEmpty(inputChannels[1].getInputChannel()); - verifyBufferOrEvent(inputGate, true, 0); - verifyBufferOrEvent(inputGate, true, 1); - verifyBufferOrEvent(inputGate, true, 0); - verifyBufferOrEvent(inputGate, false, 1); - verifyBufferOrEvent(inputGate, false, 0); + verifyBufferOrEvent(inputGate, true, 0, true); + verifyBufferOrEvent(inputGate, true, 1, true); + verifyBufferOrEvent(inputGate, true, 0, true); + verifyBufferOrEvent(inputGate, false, 1, true); + verifyBufferOrEvent(inputGate, false, 0, false); // Return null when the input gate has received all end-of-partition events assertTrue(inputGate.isFinished()); - assertFalse(inputGate.getNextBufferOrEvent().isPresent()); + } + + @Test(timeout = 120 * 1000) + public void testIsMoreAvailableReadingFromSingleInputChannel() throws Exception { + // Setup + final SingleInputGate inputGate = createInputGate(); + + final TestInputChannel[] inputChannels = new TestInputChannel[]{ + new TestInputChannel(inputGate, 0), + new TestInputChannel(inputGate, 1) + }; + + inputGate.setInputChannel( + new IntermediateResultPartitionID(), inputChannels[0].getInputChannel()); + + inputGate.setInputChannel( + new IntermediateResultPartitionID(), inputChannels[1].getInputChannel()); + + // Test + inputChannels[0].readBuffer(); + inputChannels[0].readBuffer(false); + + inputGate.notifyChannelNonEmpty(inputChannels[0].getInputChannel()); + + verifyBufferOrEvent(inputGate, true, 0, true); + verifyBufferOrEvent(inputGate, true, 0, false); } @Test @@ -428,13 +453,28 @@ private static SingleInputGate createInputGate(int numberOfInputChannels) { } static void verifyBufferOrEvent( - InputGate inputGate, - boolean isBuffer, - int channelIndex) throws IOException, InterruptedException { + InputGate inputGate, + boolean expectedIsBuffer, + int expectedChannelIndex, + boolean expectedMoreAvailable) throws IOException, InterruptedException { final Optional bufferOrEvent = inputGate.getNextBufferOrEvent(); assertTrue(bufferOrEvent.isPresent()); - assertEquals(isBuffer, bufferOrEvent.get().isBuffer()); - assertEquals(channelIndex, bufferOrEvent.get().getChannelIndex()); + assertEquals(expectedIsBuffer, bufferOrEvent.get().isBuffer()); + assertEquals(expectedChannelIndex, bufferOrEvent.get().getChannelIndex()); + assertEquals(expectedMoreAvailable, bufferOrEvent.get().moreAvailable()); + if (!expectedMoreAvailable) { + try { + assertFalse(inputGate.pollNextBufferOrEvent().isPresent()); + } + catch (UnsupportedOperationException ex) { + /** + * {@link UnionInputGate#pollNextBufferOrEvent()} is unsupported at the moment. + */ + if (!(inputGate instanceof UnionInputGate)) { + throw ex; + } + } + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java index f9060f3689320..3ae3a8a2f69d6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java @@ -56,20 +56,28 @@ public TestInputChannel(SingleInputGate inputGate, int channelIndex) { } public TestInputChannel read(Buffer buffer) throws IOException, InterruptedException { + return read(buffer, true); + } + + public TestInputChannel read(Buffer buffer, boolean moreAvailable) throws IOException, InterruptedException { if (stubbing == null) { - stubbing = when(mock.getNextBuffer()).thenReturn(Optional.of(new BufferAndAvailability(buffer, true, 0))); + stubbing = when(mock.getNextBuffer()).thenReturn(Optional.of(new BufferAndAvailability(buffer, moreAvailable, 0))); } else { - stubbing = stubbing.thenReturn(Optional.of(new BufferAndAvailability(buffer, true, 0))); + stubbing = stubbing.thenReturn(Optional.of(new BufferAndAvailability(buffer, moreAvailable, 0))); } return this; } public TestInputChannel readBuffer() throws IOException, InterruptedException { + return readBuffer(true); + } + + public TestInputChannel readBuffer(boolean moreAvailable) throws IOException, InterruptedException { final Buffer buffer = mock(Buffer.class); when(buffer.isBuffer()).thenReturn(true); - return read(buffer); + return read(buffer, moreAvailable); } public TestInputChannel readEndOfPartitionEvent() throws IOException, InterruptedException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java index 9b164710afd58..912cd5b24347f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGateTest.java @@ -26,6 +26,7 @@ import org.junit.Test; +import static org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateTest.verifyBufferOrEvent; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -93,22 +94,22 @@ testTaskName, new JobID(), ig2.notifyChannelNonEmpty(inputChannels[1][3].getInputChannel()); ig2.notifyChannelNonEmpty(inputChannels[1][4].getInputChannel()); - SingleInputGateTest.verifyBufferOrEvent(union, true, 0); // gate 1, channel 0 - SingleInputGateTest.verifyBufferOrEvent(union, true, 3); // gate 2, channel 0 - SingleInputGateTest.verifyBufferOrEvent(union, true, 1); // gate 1, channel 1 - SingleInputGateTest.verifyBufferOrEvent(union, true, 4); // gate 2, channel 1 - SingleInputGateTest.verifyBufferOrEvent(union, true, 2); // gate 1, channel 2 - SingleInputGateTest.verifyBufferOrEvent(union, true, 5); // gate 2, channel 1 - SingleInputGateTest.verifyBufferOrEvent(union, false, 0); // gate 1, channel 0 - SingleInputGateTest.verifyBufferOrEvent(union, true, 6); // gate 2, channel 1 - SingleInputGateTest.verifyBufferOrEvent(union, false, 1); // gate 1, channel 1 - SingleInputGateTest.verifyBufferOrEvent(union, true, 7); // gate 2, channel 1 - SingleInputGateTest.verifyBufferOrEvent(union, false, 2); // gate 1, channel 2 - SingleInputGateTest.verifyBufferOrEvent(union, false, 3); // gate 2, channel 0 - SingleInputGateTest.verifyBufferOrEvent(union, false, 4); // gate 2, channel 1 - SingleInputGateTest.verifyBufferOrEvent(union, false, 5); // gate 2, channel 2 - SingleInputGateTest.verifyBufferOrEvent(union, false, 6); // gate 2, channel 3 - SingleInputGateTest.verifyBufferOrEvent(union, false, 7); // gate 2, channel 4 + verifyBufferOrEvent(union, true, 0, true); // gate 1, channel 0 + verifyBufferOrEvent(union, true, 3, true); // gate 2, channel 0 + verifyBufferOrEvent(union, true, 1, true); // gate 1, channel 1 + verifyBufferOrEvent(union, true, 4, true); // gate 2, channel 1 + verifyBufferOrEvent(union, true, 2, true); // gate 1, channel 2 + verifyBufferOrEvent(union, true, 5, true); // gate 2, channel 1 + verifyBufferOrEvent(union, false, 0, true); // gate 1, channel 0 + verifyBufferOrEvent(union, true, 6, true); // gate 2, channel 1 + verifyBufferOrEvent(union, false, 1, true); // gate 1, channel 1 + verifyBufferOrEvent(union, true, 7, true); // gate 2, channel 1 + verifyBufferOrEvent(union, false, 2, true); // gate 1, channel 2 + verifyBufferOrEvent(union, false, 3, true); // gate 2, channel 0 + verifyBufferOrEvent(union, false, 4, true); // gate 2, channel 1 + verifyBufferOrEvent(union, false, 5, true); // gate 2, channel 2 + verifyBufferOrEvent(union, false, 6, true); // gate 2, channel 3 + verifyBufferOrEvent(union, false, 7, false); // gate 2, channel 4 // Return null when the input gate has received all end-of-partition events assertTrue(union.isFinished()); From 8e62f90739e2319491df983917dc7ab484de2550 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 23 Feb 2018 11:27:54 +0100 Subject: [PATCH 018/268] [hotfix][tests] Do not hide original exception in SuccessAfterNetworkBuffersFailureITCase (cherry picked from commit 2c2e189) --- ...ccessAfterNetworkBuffersFailureITCase.java | 40 +++++-------------- 1 file changed, 11 insertions(+), 29 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java index dc19ad1d64712..dbd0f7976d6b6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java @@ -65,38 +65,20 @@ private static Configuration getConfiguration() { } @Test - public void testSuccessfulProgramAfterFailure() { + public void testSuccessfulProgramAfterFailure() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + runConnectedComponents(env); + try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); - - try { - runConnectedComponents(env); - } - catch (Exception e) { - e.printStackTrace(); - fail("Program Execution should have succeeded."); - } - - try { - runKMeans(env); - fail("This program execution should have failed."); - } - catch (JobExecutionException e) { - assertTrue(e.getCause().getMessage().contains("Insufficient number of network buffers")); - } - - try { - runConnectedComponents(env); - } - catch (Exception e) { - e.printStackTrace(); - fail("Program Execution should have succeeded."); - } + runKMeans(env); + fail("This program execution should have failed."); } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + catch (JobExecutionException e) { + assertTrue(e.getCause().getMessage().contains("Insufficient number of network buffers")); } + + runConnectedComponents(env); } private static void runConnectedComponents(ExecutionEnvironment env) throws Exception { From 8eb6a30798c09d171e3eb8019b53e677252bd5ba Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 23 Feb 2018 11:28:20 +0100 Subject: [PATCH 019/268] [FLINK-8694][runtime] Fix notifyDataAvailable race condition Before there was a race condition that might resulted in igonoring some notifyDataAvailable calls. This fixes the problem by moving buffersAvailable handling to Supartitions and adds stress test for flushAlways (without this fix this test is dead locking). (cherry picked from commit ebd39f3) --- ...reditBasedSequenceNumberingViewReader.java | 10 +--- .../netty/SequenceNumberingViewReader.java | 7 +-- .../partition/PipelinedSubpartition.java | 37 +++++++++++-- .../partition/PipelinedSubpartitionView.java | 5 ++ .../partition/ResultSubpartitionView.java | 2 + .../partition/SpillableSubpartition.java | 1 - .../partition/SpillableSubpartitionView.java | 28 +++++++--- .../partition/SpilledSubpartitionView.java | 8 +++ .../buffer/BufferBuilderTestUtils.java | 4 ++ .../netty/CancelPartitionRequestTest.java | 5 ++ .../netty/PartitionRequestQueueTest.java | 26 +++++++-- .../partition/PipelinedSubpartitionTest.java | 53 +++++++++++++++++++ .../partition/SpillableSubpartitionTest.java | 9 ++-- .../partition/SubpartitionTestBase.java | 5 ++ ...StreamNetworkThroughputBenchmarkTests.java | 8 +++ 15 files changed, 173 insertions(+), 35 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java index d02b2bf1ad5ad..9acbbacf2735b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel; import java.io.IOException; -import java.util.concurrent.atomic.AtomicBoolean; /** * Simple wrapper for the subpartition view used in the new network credit-based mode. @@ -44,8 +43,6 @@ class CreditBasedSequenceNumberingViewReader implements BufferAvailabilityListen private final InputChannelID receiverId; - private final AtomicBoolean buffersAvailable = new AtomicBoolean(); - private final PartitionRequestQueue requestQueue; private volatile ResultSubpartitionView subpartitionView; @@ -118,7 +115,7 @@ public boolean isRegisteredAsAvailable() { @Override public boolean isAvailable() { // BEWARE: this must be in sync with #isAvailable()! - return buffersAvailable.get() && + return hasBuffersAvailable() && (numCreditsAvailable > 0 || subpartitionView.nextBufferIsEvent()); } @@ -154,14 +151,13 @@ int getNumCreditsAvailable() { @VisibleForTesting boolean hasBuffersAvailable() { - return buffersAvailable.get(); + return subpartitionView.isAvailable(); } @Override public BufferAndAvailability getNextBuffer() throws IOException, InterruptedException { BufferAndBacklog next = subpartitionView.getNextBuffer(); if (next != null) { - buffersAvailable.set(next.isMoreAvailable()); sequenceNumber++; if (next.buffer().isBuffer() && --numCreditsAvailable < 0) { @@ -197,7 +193,6 @@ public void releaseAllResources() throws IOException { @Override public void notifyDataAvailable() { - buffersAvailable.set(true); requestQueue.notifyReaderNonEmpty(this); } @@ -206,7 +201,6 @@ public String toString() { return "CreditBasedSequenceNumberingViewReader{" + "requestLock=" + requestLock + ", receiverId=" + receiverId + - ", buffersAvailable=" + buffersAvailable.get() + ", sequenceNumber=" + sequenceNumber + ", numCreditsAvailable=" + numCreditsAvailable + ", isRegisteredAsAvailable=" + isRegisteredAsAvailable + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java index 2d9635ceaeb65..6a83af1383782 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel; import java.io.IOException; -import java.util.concurrent.atomic.AtomicBoolean; /** * Simple wrapper for the subpartition view used in the old network mode. @@ -43,8 +42,6 @@ class SequenceNumberingViewReader implements BufferAvailabilityListener, Network private final InputChannelID receiverId; - private final AtomicBoolean buffersAvailable = new AtomicBoolean(); - private final PartitionRequestQueue requestQueue; private volatile ResultSubpartitionView subpartitionView; @@ -96,7 +93,7 @@ public boolean isRegisteredAsAvailable() { @Override public boolean isAvailable() { - return buffersAvailable.get(); + return subpartitionView.isAvailable(); } @Override @@ -113,7 +110,6 @@ public int getSequenceNumber() { public BufferAndAvailability getNextBuffer() throws IOException, InterruptedException { BufferAndBacklog next = subpartitionView.getNextBuffer(); if (next != null) { - buffersAvailable.set(next.isMoreAvailable()); sequenceNumber++; return new BufferAndAvailability(next.buffer(), next.isMoreAvailable(), next.buffersInBacklog()); } else { @@ -143,7 +139,6 @@ public void releaseAllResources() throws IOException { @Override public void notifyDataAvailable() { - buffersAvailable.set(true); requestQueue.notifyReaderNonEmpty(this); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java index a9c6e57b96597..cc7936350479f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java @@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; import java.io.IOException; @@ -48,6 +49,9 @@ class PipelinedSubpartition extends ResultSubpartition { /** Flag indicating whether the subpartition has been finished. */ private boolean isFinished; + @GuardedBy("buffers") + private boolean flushRequested; + /** Flag indicating whether the subpartition has been released. */ private volatile boolean isReleased; @@ -65,9 +69,11 @@ public boolean add(BufferConsumer bufferConsumer) { @Override public void flush() { synchronized (buffers) { - if (readView != null) { - readView.notifyDataAvailable(); + if (buffers.isEmpty()) { + return; } + flushRequested = !buffers.isEmpty(); + notifyDataAvailable(); } } @@ -93,7 +99,7 @@ private boolean add(BufferConsumer bufferConsumer, boolean finish) { if (finish) { isFinished = true; - notifyDataAvailable(); + flush(); } else { maybeNotifyDataAvailable(); @@ -138,17 +144,28 @@ BufferAndBacklog pollBuffer() { synchronized (buffers) { Buffer buffer = null; + if (buffers.isEmpty()) { + flushRequested = false; + } + while (!buffers.isEmpty()) { BufferConsumer bufferConsumer = buffers.peek(); buffer = bufferConsumer.build(); + checkState(bufferConsumer.isFinished() || buffers.size() == 1, "When there are multiple buffers, an unfinished bufferConsumer can not be at the head of the buffers queue."); + if (buffers.size() == 1) { + // turn off flushRequested flag if we drained all of the available data + flushRequested = false; + } + if (bufferConsumer.isFinished()) { buffers.pop().close(); decreaseBuffersInBacklogUnsafe(bufferConsumer.isBuffer()); } + if (buffer.readableBytes() > 0) { break; } @@ -169,7 +186,7 @@ BufferAndBacklog pollBuffer() { // will be 2 or more. return new BufferAndBacklog( buffer, - getNumberOfFinishedBuffers() > 0, + isAvailableUnsafe(), getBuffersInBacklog(), _nextBufferIsEvent()); } @@ -211,13 +228,23 @@ public PipelinedSubpartitionView createReadView(BufferAvailabilityListener avail readView = new PipelinedSubpartitionView(this, availabilityListener); if (!buffers.isEmpty()) { - readView.notifyDataAvailable(); + notifyDataAvailable(); } } return readView; } + public boolean isAvailable() { + synchronized (buffers) { + return isAvailableUnsafe(); + } + } + + private boolean isAvailableUnsafe() { + return flushRequested || getNumberOfFinishedBuffers() > 0; + } + // ------------------------------------------------------------------------ int getCurrentNumberOfBuffers() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java index c60a604f0d7b9..9d083585b8273 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionView.java @@ -80,6 +80,11 @@ public boolean nextBufferIsEvent() { return parent.nextBufferIsEvent(); } + @Override + public boolean isAvailable() { + return parent.isAvailable(); + } + @Override public Throwable getFailureCause() { return parent.getFailureCause(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java index 41fbb0a63c669..b1ccd63470475 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java @@ -57,4 +57,6 @@ public interface ResultSubpartitionView { * Returns whether the next buffer is an event or not. */ boolean nextBufferIsEvent(); + + boolean isAvailable(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java index 6ac493e7e2752..6b731d42da933 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java @@ -209,7 +209,6 @@ public ResultSubpartitionView createReadView(BufferAvailabilityListener availabi parent.getBufferProvider().getMemorySegmentSize(), availabilityListener); } - return readView; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java index b821dcf6afe8d..3c73e43d8cb9f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java @@ -153,18 +153,16 @@ public BufferAndBacklog getNextBuffer() throws IOException, InterruptedException return null; } else if (nextBuffer != null) { current = nextBuffer.build(); + checkState(nextBuffer.isFinished(), + "We can only read from SpillableSubpartition after it was finished"); - if (nextBuffer.isFinished()) { - newBacklog = parent.decreaseBuffersInBacklogUnsafe(nextBuffer.isBuffer()); - nextBuffer.close(); - nextBuffer = buffers.poll(); - } + newBacklog = parent.decreaseBuffersInBacklogUnsafe(nextBuffer.isBuffer()); + nextBuffer.close(); + nextBuffer = buffers.poll(); - isMoreAvailable = buffers.size() > 0; if (nextBuffer != null) { - isMoreAvailable = true; - listener.notifyDataAvailable(); nextBufferIsEvent = !nextBuffer.isBuffer(); + isMoreAvailable = true; } parent.updateStatistics(current); @@ -245,6 +243,20 @@ public boolean nextBufferIsEvent() { return spilledView.nextBufferIsEvent(); } + @Override + public boolean isAvailable() { + synchronized (buffers) { + if (nextBuffer != null) { + return true; + } + else if (spilledView == null) { + return false; + } + } // else: spilled + + return spilledView.isAvailable(); + } + @Override public Throwable getFailureCause() { SpilledSubpartitionView spilled = spilledView; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java index 4c5cd2e0e49ce..378b0867d6fee 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java @@ -219,6 +219,14 @@ public boolean nextBufferIsEvent() { } } + @Override + public synchronized boolean isAvailable() { + if (nextBuffer != null) { + return true; + } + return !fileReader.hasReachedEndOfFile(); + } + @Override public Throwable getFailureCause() { return parent.getFailureCause(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java index a6e9fdcd0b34d..7beb18fd21fca 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderTestUtils.java @@ -38,6 +38,10 @@ public static BufferBuilder createBufferBuilder(int size) { return createFilledBufferBuilder(size, 0); } + public static BufferBuilder createFilledBufferBuilder(int dataSize) { + return createFilledBufferBuilder(BUFFER_SIZE, dataSize); + } + public static BufferBuilder createFilledBufferBuilder(int size, int dataSize) { checkArgument(size >= dataSize); BufferBuilder bufferBuilder = new BufferBuilder( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java index 56abff1a61463..eca8263eabd9c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CancelPartitionRequestTest.java @@ -219,6 +219,11 @@ public boolean nextBufferIsEvent() { return false; } + @Override + public boolean isAvailable() { + return true; + } + @Override public Throwable getFailureCause() { return null; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java index 16418ff995938..f614c18902be1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueueTest.java @@ -66,7 +66,7 @@ public void testNotifyReaderNonEmptyOnEmptyReaders() throws Exception { CreditBasedSequenceNumberingViewReader reader1 = new CreditBasedSequenceNumberingViewReader(new InputChannelID(0, 0), 10, queue); CreditBasedSequenceNumberingViewReader reader2 = new CreditBasedSequenceNumberingViewReader(new InputChannelID(1, 1), 10, queue); - reader1.requestSubpartitionView((partitionId, index, availabilityListener) -> new NotReleasedResultSubpartitionView(), new ResultPartitionID(), 0); + reader1.requestSubpartitionView((partitionId, index, availabilityListener) -> new EmptyAlwaysAvailableResultSubpartitionView(), new ResultPartitionID(), 0); reader1.notifyDataAvailable(); assertTrue(reader1.isAvailable()); assertFalse(reader1.isRegisteredAsAvailable()); @@ -178,6 +178,11 @@ public BufferAndBacklog getNextBuffer() { buffers, false); } + + @Override + public boolean isAvailable() { + return buffersInBacklog.get() > 0; + } } private static class ReadOnlyBufferResultSubpartitionView extends DefaultBufferResultSubpartitionView { @@ -197,14 +202,19 @@ public BufferAndBacklog getNextBuffer() { } } - private static class NotReleasedResultSubpartitionView extends NoOpResultSubpartitionView { + private static class EmptyAlwaysAvailableResultSubpartitionView extends NoOpResultSubpartitionView { @Override public boolean isReleased() { return false; } + + @Override + public boolean isAvailable() { + return true; + } } - private static class ReleasedResultSubpartitionView extends NoOpResultSubpartitionView { + private static class ReleasedResultSubpartitionView extends EmptyAlwaysAvailableResultSubpartitionView { @Override public boolean isReleased() { return true; @@ -263,6 +273,11 @@ private static class NextIsEventResultSubpartitionView extends NoOpResultSubpart public boolean nextBufferIsEvent() { return true; } + + @Override + public boolean isAvailable() { + return true; + } } /** @@ -387,5 +402,10 @@ public Throwable getFailureCause() { public boolean nextBufferIsEvent() { return false; } + + @Override + public boolean isAvailable() { + return false; + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index 528f0e296d341..ee678abc4ccc4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -43,6 +43,7 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createBufferBuilder; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createEventBufferConsumer; +import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferBuilder; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; import static org.apache.flink.runtime.io.network.util.TestBufferFactory.BUFFER_SIZE; import static org.apache.flink.util.FutureUtil.waitForAll; @@ -142,6 +143,48 @@ public void testAddNonEmptyNotFinishedBuffer() throws Exception { } } + /** + * Normally moreAvailable flag from InputChannel should ignore non finished BufferConsumers, otherwise we would + * busy loop on the unfinished BufferConsumers. + */ + @Test + public void testUnfinishedBufferBehindFinished() throws Exception { + final ResultSubpartition subpartition = createSubpartition(); + AwaitableBufferAvailablityListener availablityListener = new AwaitableBufferAvailablityListener(); + ResultSubpartitionView readView = subpartition.createReadView(availablityListener); + + try { + subpartition.add(createFilledBufferConsumer(1025)); // finished + subpartition.add(createFilledBufferBuilder(1024).createBufferConsumer()); // not finished + + assertNextBuffer(readView, 1025, false, 1); + } finally { + subpartition.release(); + } + } + + /** + * After flush call unfinished BufferConsumers should be reported as available, otherwise we might not flush some + * of the data. + */ + @Test + public void testFlushWithUnfinishedBufferBehindFinished() throws Exception { + final ResultSubpartition subpartition = createSubpartition(); + AwaitableBufferAvailablityListener availablityListener = new AwaitableBufferAvailablityListener(); + ResultSubpartitionView readView = subpartition.createReadView(availablityListener); + + try { + subpartition.add(createFilledBufferConsumer(1025)); // finished + subpartition.add(createFilledBufferBuilder(1024).createBufferConsumer()); // not finished + subpartition.flush(); + + assertNextBuffer(readView, 1025, true, 1); + assertNextBuffer(readView, 1024, false, 1); + } finally { + subpartition.release(); + } + } + @Test public void testMultipleEmptyBuffers() throws Exception { final ResultSubpartition subpartition = createSubpartition(); @@ -187,6 +230,16 @@ public void testIllegalReadViewRequest() throws Exception { } } + @Test + public void testEmptyFlush() throws Exception { + final PipelinedSubpartition subpartition = createSubpartition(); + + AwaitableBufferAvailablityListener listener = new AwaitableBufferAvailablityListener(); + subpartition.createReadView(listener); + subpartition.flush(); + assertEquals(0, listener.getNumNotifications()); + } + @Test public void testBasicPipelinedProduceConsumeLogic() throws Exception { final PipelinedSubpartition subpartition = createSubpartition(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java index a6be748194ba7..e41a85c5207b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java @@ -53,7 +53,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; @@ -319,7 +318,8 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception assertEquals(2, partition.getBuffersInBacklog()); assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); read.buffer().recycleBuffer(); - assertEquals(2, listener.getNumNotifications()); + assertTrue(read.isMoreAvailable()); + assertEquals(1, listener.getNumNotifications()); // since isMoreAvailable is set to true, no need for notification assertFalse(bufferConsumer.isRecycled()); assertFalse(read.nextBufferIsEvent()); @@ -332,8 +332,9 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception // only updated when getting/spilling the buffers but without the nextBuffer (kept in memory) assertEquals(BUFFER_DATA_SIZE * 3 + 4, partition.getTotalNumberOfBytes()); - listener.awaitNotifications(3, 30_000); - assertEquals(3, listener.getNumNotifications()); + listener.awaitNotifications(2, 30_000); + // Spiller finished + assertEquals(2, listener.getNumNotifications()); assertFalse(reader.nextBufferIsEvent()); // second buffer (retained in SpillableSubpartition#nextBuffer) read = reader.getNextBuffer(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index 1b861dfab51db..215726b3b5ad1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -30,6 +30,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; @@ -143,4 +144,8 @@ protected void assertNextBuffer( assertEquals(expectedIsMoreAvailable, bufferAndBacklog.isMoreAvailable()); assertEquals(expectedBuffersInBacklog, bufferAndBacklog.buffersInBacklog()); } + + protected void assertNoNextBuffer(ResultSubpartitionView readView) throws IOException, InterruptedException { + assertNull(readView.getNextBuffer()); + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTests.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTests.java index a8251a80da61a..a60fa3c4988ba 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTests.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTests.java @@ -52,6 +52,14 @@ public void largeRemoteMode() throws Exception { env.tearDown(); } + @Test + public void largeRemoteAlwaysFlush() throws Exception { + StreamNetworkThroughputBenchmark env = new StreamNetworkThroughputBenchmark(); + env.setUp(1, 1, 0, false); + env.executeBenchmark(1_000_000); + env.tearDown(); + } + @Test public void pointToMultiPointBenchmark() throws Exception { StreamNetworkThroughputBenchmark benchmark = new StreamNetworkThroughputBenchmark(); From 61a34a691e7d5233f18ac72a1ab8fb09b53c4753 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 26 Feb 2018 16:13:06 +0100 Subject: [PATCH 020/268] [FLINK-8805][runtime] Optimize EvenSerializer.isEvent method For example, previously if the method was used to check for EndOfPartitionEvent and the Buffer contained huge custom event, the even had to be deserialized before performing the actual check. Now we are quickly entering the correct if/else branch and doing full costly deserialization only if we have to. Other calls to isEvent() then checking against EndOfPartitionEvent were not used. (cherry picked from commit 767027f) --- .../api/serialization/EventSerializer.java | 57 +++++-------------- .../network/netty/PartitionRequestQueue.java | 3 +- .../serialization/EventSerializerTest.java | 39 ++++++++----- 3 files changed, 42 insertions(+), 57 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java index d7fb7e851c150..8d76bb26837f3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java @@ -101,16 +101,15 @@ else if (eventClass == CancelCheckpointMarker.class) { } /** - * Identifies whether the given buffer encodes the given event. + * Identifies whether the given buffer encodes the given event. Custom events are not supported. * *

    Pre-condition: This buffer must encode some event!

    * * @param buffer the buffer to peak into * @param eventClass the expected class of the event type - * @param classLoader the class loader to use for custom event classes * @return whether the event class of the buffer matches the given eventClass */ - private static boolean isEvent(ByteBuffer buffer, Class eventClass, ClassLoader classLoader) throws IOException { + private static boolean isEvent(ByteBuffer buffer, Class eventClass) throws IOException { if (buffer.remaining() < 4) { throw new IOException("Incomplete event"); } @@ -122,38 +121,16 @@ private static boolean isEvent(ByteBuffer buffer, Class eventClass, ClassLoad try { int type = buffer.getInt(); - switch (type) { - case END_OF_PARTITION_EVENT: - return eventClass.equals(EndOfPartitionEvent.class); - case CHECKPOINT_BARRIER_EVENT: - return eventClass.equals(CheckpointBarrier.class); - case END_OF_SUPERSTEP_EVENT: - return eventClass.equals(EndOfSuperstepEvent.class); - case CANCEL_CHECKPOINT_MARKER_EVENT: - return eventClass.equals(CancelCheckpointMarker.class); - case OTHER_EVENT: - try { - final DataInputDeserializer deserializer = new DataInputDeserializer(buffer); - final String className = deserializer.readUTF(); - - final Class clazz; - try { - clazz = classLoader.loadClass(className).asSubclass(AbstractEvent.class); - } - catch (ClassNotFoundException e) { - throw new IOException("Could not load event class '" + className + "'.", e); - } - catch (ClassCastException e) { - throw new IOException("The class '" + className + "' is not a valid subclass of '" - + AbstractEvent.class.getName() + "'.", e); - } - return eventClass.equals(clazz); - } - catch (Exception e) { - throw new IOException("Error while deserializing or instantiating event.", e); - } - default: - throw new IOException("Corrupt byte stream for event"); + if (eventClass.equals(EndOfPartitionEvent.class)) { + return type == END_OF_PARTITION_EVENT; + } else if (eventClass.equals(CheckpointBarrier.class)) { + return type == CHECKPOINT_BARRIER_EVENT; + } else if (eventClass.equals(EndOfSuperstepEvent.class)) { + return type == END_OF_SUPERSTEP_EVENT; + } else if (eventClass.equals(CancelCheckpointMarker.class)) { + return type == CANCEL_CHECKPOINT_MARKER_EVENT; + } else { + throw new UnsupportedOperationException("Unsupported eventClass = " + eventClass); } } finally { @@ -314,17 +291,13 @@ public static AbstractEvent fromBuffer(Buffer buffer, ClassLoader classLoader) t } /** - * Identifies whether the given buffer encodes the given event. + * Identifies whether the given buffer encodes the given event. Custom events are not supported. * * @param buffer the buffer to peak into * @param eventClass the expected class of the event type - * @param classLoader the class loader to use for custom event classes * @return whether the event class of the buffer matches the given eventClass */ - public static boolean isEvent(final Buffer buffer, - final Class eventClass, - final ClassLoader classLoader) throws IOException { - return !buffer.isBuffer() && - isEvent(buffer.getNioBufferReadable(), eventClass, classLoader); + public static boolean isEvent(Buffer buffer, Class eventClass) throws IOException { + return !buffer.isBuffer() && isEvent(buffer.getNioBufferReadable(), eventClass); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java index 8d43815ada67a..d63a88e718276 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java @@ -287,8 +287,7 @@ private NetworkSequenceViewReader pollAvailableReader() { } private boolean isEndOfPartitionEvent(Buffer buffer) throws IOException { - return EventSerializer.isEvent(buffer, EndOfPartitionEvent.class, - getClass().getClassLoader()); + return EventSerializer.isEvent(buffer, EndOfPartitionEvent.class); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java index de5f4a82a7a93..c00fea7ebe193 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializerTest.java @@ -33,11 +33,13 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for the {@link EventSerializer}. @@ -95,7 +97,7 @@ public void testSerializeDeserializeEvent() throws Exception { } /** - * Tests {@link EventSerializer#isEvent(Buffer, Class, ClassLoader)} + * Tests {@link EventSerializer#isEvent(Buffer, Class)} * whether it peaks into the buffer only, i.e. after the call, the buffer * is still de-serializable. */ @@ -106,8 +108,7 @@ public void testIsEventPeakOnly() throws Exception { try { final ClassLoader cl = getClass().getClassLoader(); assertTrue( - EventSerializer - .isEvent(serializedEvent, EndOfPartitionEvent.class, cl)); + EventSerializer.isEvent(serializedEvent, EndOfPartitionEvent.class)); EndOfPartitionEvent event = (EndOfPartitionEvent) EventSerializer .fromBuffer(serializedEvent, cl); assertEquals(EndOfPartitionEvent.INSTANCE, event); @@ -117,7 +118,7 @@ public void testIsEventPeakOnly() throws Exception { } /** - * Tests {@link EventSerializer#isEvent(Buffer, Class, ClassLoader)} returns + * Tests {@link EventSerializer#isEvent(Buffer, Class)} returns * the correct answer for various encoded event buffers. */ @Test @@ -130,12 +131,25 @@ public void testIsEvent() throws Exception { new CancelCheckpointMarker(287087987329842L) }; + Class[] expectedClasses = Arrays.stream(events) + .map(AbstractEvent::getClass) + .toArray(Class[]::new); + for (AbstractEvent evt : events) { - for (AbstractEvent evt2 : events) { - if (evt == evt2) { - assertTrue(checkIsEvent(evt, evt2.getClass())); + for (Class expectedClass: expectedClasses) { + if (expectedClass.equals(TestTaskEvent.class)) { + try { + checkIsEvent(evt, expectedClass); + fail("This should fail"); + } + catch (UnsupportedOperationException ex) { + // expected + } + } + else if (evt.getClass().equals(expectedClass)) { + assertTrue(checkIsEvent(evt, expectedClass)); } else { - assertFalse(checkIsEvent(evt, evt2.getClass())); + assertFalse(checkIsEvent(evt, expectedClass)); } } } @@ -143,23 +157,22 @@ public void testIsEvent() throws Exception { /** * Returns the result of - * {@link EventSerializer#isEvent(Buffer, Class, ClassLoader)} on a buffer + * {@link EventSerializer#isEvent(Buffer, Class)} on a buffer * that encodes the given event. * * @param event the event to encode * @param eventClass the event class to check against * - * @return whether {@link EventSerializer#isEvent(ByteBuffer, Class, ClassLoader)} + * @return whether {@link EventSerializer#isEvent(ByteBuffer, Class)} * thinks the encoded buffer matches the class */ private boolean checkIsEvent( AbstractEvent event, - Class eventClass) throws IOException { + Class eventClass) throws IOException { final Buffer serializedEvent = EventSerializer.toBuffer(event); try { - final ClassLoader cl = getClass().getClassLoader(); - return EventSerializer.isEvent(serializedEvent, eventClass, cl); + return EventSerializer.isEvent(serializedEvent, eventClass); } finally { serializedEvent.recycleBuffer(); } From d5338c4154e5de029b3b30e3ef0a0732bf7f68e7 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Tue, 27 Feb 2018 10:39:00 +0100 Subject: [PATCH 021/268] [FLINK-8750][runtime] Improve detection of no remaining data after EndOfPartitionEvent Because of race condition between: 1. releasing inputChannelsWithData lock in this method and reaching this place 2. empty data notification that re-enqueues a channel we can end up with moreAvailable flag set to true, while we expect no more data. This commit detects such situation, makes a correct assertion and turn off moreAvailable flag. (cherry picked from commit b9b7416) --- .../io/network/partition/consumer/SingleInputGate.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index be4035c55b173..b9091b2e52d15 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -554,6 +554,12 @@ private Optional getNextBufferOrEvent(boolean blocking) throws IO channelsWithEndOfPartitionEvents.set(currentChannel.getChannelIndex()); if (channelsWithEndOfPartitionEvents.cardinality() == numberOfInputChannels) { + // Because of race condition between: + // 1. releasing inputChannelsWithData lock in this method and reaching this place + // 2. empty data notification that re-enqueues a channel + // we can end up with moreAvailable flag set to true, while we expect no more data. + checkState(!moreAvailable || !pollNextBufferOrEvent().isPresent()); + moreAvailable = false; hasReceivedAllEndOfPartitionEvents = true; } From 32384ed9b00cf0e1961d355dc4080f25a2156e58 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Thu, 22 Feb 2018 15:41:38 +0100 Subject: [PATCH 022/268] [FLINK-8747][bugfix] The tag of waiting for floating buffers in RemoteInputChannel should be updated properly (cherry picked from commit 6e9e0dd) --- .../consumer/RemoteInputChannel.java | 6 +++ .../consumer/RemoteInputChannelTest.java | 42 ++++++++++++++----- 2 files changed, 37 insertions(+), 11 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index 8174359db0276..990166f0b1eff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -337,6 +337,11 @@ public int getSenderBacklog() { return numRequiredBuffers - initialCredit; } + @VisibleForTesting + public boolean isWaitingForFloatingBuffers() { + return isWaitingForFloatingBuffers; + } + /** * The Buffer pool notifies this channel of an available floating buffer. If the channel is released or * currently does not need extra buffers, the buffer should be recycled to the buffer pool. Otherwise, @@ -362,6 +367,7 @@ public boolean notifyBufferAvailable(Buffer buffer) { // Important: double check the isReleased state inside synchronized block, so there is no // race condition when notifyBufferAvailable and releaseAllResources running in parallel. if (isReleased.get() || bufferQueue.getAvailableBufferSize() >= numRequiredBuffers) { + isWaitingForFloatingBuffers = false; buffer.recycleBuffer(); return false; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index 7c8ed18d94937..e3e6623c929ad 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -357,6 +357,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 16, inputChannel.getNumberOfRequiredBuffers()); assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); + assertTrue(inputChannel.isWaitingForFloatingBuffers()); // Increase the backlog inputChannel.onSenderBacklog(16); @@ -370,11 +371,12 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 18, inputChannel.getNumberOfRequiredBuffers()); assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); + assertTrue(inputChannel.isWaitingForFloatingBuffers()); - // Recycle one floating buffer - floatingBufferQueue.poll().recycleBuffer(); + // Recycle one exclusive buffer + exclusiveBuffer.recycleBuffer(); - // Assign the floating buffer to the listener and the channel is still waiting for more floating buffers + // The exclusive buffer is returned to the channel directly verify(bufferPool, times(15)).requestBuffer(); verify(bufferPool, times(1)).addBufferListener(inputChannel); assertEquals("There should be 14 buffers available in the channel", @@ -383,8 +385,9 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 18, inputChannel.getNumberOfRequiredBuffers()); assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); + assertTrue(inputChannel.isWaitingForFloatingBuffers()); - // Recycle one more floating buffer + // Recycle one floating buffer floatingBufferQueue.poll().recycleBuffer(); // Assign the floating buffer to the listener and the channel is still waiting for more floating buffers @@ -396,32 +399,49 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 18, inputChannel.getNumberOfRequiredBuffers()); assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); + assertTrue(inputChannel.isWaitingForFloatingBuffers()); // Decrease the backlog - inputChannel.onSenderBacklog(15); + inputChannel.onSenderBacklog(13); // Only the number of required buffers is changed by (backlog + numExclusiveBuffers) verify(bufferPool, times(15)).requestBuffer(); verify(bufferPool, times(1)).addBufferListener(inputChannel); assertEquals("There should be 15 buffers available in the channel", 15, inputChannel.getNumberOfAvailableBuffers()); - assertEquals("There should be 17 buffers required in the channel", - 17, inputChannel.getNumberOfRequiredBuffers()); + assertEquals("There should be 15 buffers required in the channel", + 15, inputChannel.getNumberOfRequiredBuffers()); assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); + assertTrue(inputChannel.isWaitingForFloatingBuffers()); - // Recycle one exclusive buffer - exclusiveBuffer.recycleBuffer(); + // Recycle one more floating buffer + floatingBufferQueue.poll().recycleBuffer(); - // The exclusive buffer is returned to the channel directly + // Return the floating buffer to the buffer pool and the channel is not waiting for more floating buffers verify(bufferPool, times(15)).requestBuffer(); verify(bufferPool, times(1)).addBufferListener(inputChannel); + assertEquals("There should be 15 buffers available in the channel", + 15, inputChannel.getNumberOfAvailableBuffers()); + assertEquals("There should be 15 buffers required in the channel", + 15, inputChannel.getNumberOfRequiredBuffers()); + assertEquals("There should be 1 buffers available in local pool", + 1, bufferPool.getNumberOfAvailableMemorySegments()); + assertFalse(inputChannel.isWaitingForFloatingBuffers()); + + // Increase the backlog again + inputChannel.onSenderBacklog(15); + + // The floating buffer is requested from the buffer pool and the channel is registered as listener again. + verify(bufferPool, times(17)).requestBuffer(); + verify(bufferPool, times(2)).addBufferListener(inputChannel); assertEquals("There should be 16 buffers available in the channel", 16, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 17 buffers required in the channel", 17, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffers available in local pool", + assertEquals("There should be 0 buffer available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); + assertTrue(inputChannel.isWaitingForFloatingBuffers()); } finally { // Release all the buffer resources From f1453276095c55264f7b4097d16e2987a44b3f33 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Fri, 23 Feb 2018 02:55:57 +0100 Subject: [PATCH 023/268] [hotfix] Fix package private and comments (cherry picked from commit 6165b3d) --- .../partition/consumer/RemoteInputChannel.java | 2 +- .../consumer/RemoteInputChannelTest.java | 18 +++++++++--------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index 990166f0b1eff..0f70d448020a3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -338,7 +338,7 @@ public int getSenderBacklog() { } @VisibleForTesting - public boolean isWaitingForFloatingBuffers() { + boolean isWaitingForFloatingBuffers() { return isWaitingForFloatingBuffers; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index e3e6623c929ad..97a5688773989 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -355,7 +355,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 13, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 16 buffers required in the channel", 16, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); assertTrue(inputChannel.isWaitingForFloatingBuffers()); @@ -369,7 +369,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 13, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 18 buffers required in the channel", 18, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); assertTrue(inputChannel.isWaitingForFloatingBuffers()); @@ -383,7 +383,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 14, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 18 buffers required in the channel", 18, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); assertTrue(inputChannel.isWaitingForFloatingBuffers()); @@ -397,7 +397,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 15, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 18 buffers required in the channel", 18, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); assertTrue(inputChannel.isWaitingForFloatingBuffers()); @@ -411,7 +411,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 15, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 15 buffers required in the channel", 15, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); assertTrue(inputChannel.isWaitingForFloatingBuffers()); @@ -439,7 +439,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { 16, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 17 buffers required in the channel", 17, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); assertTrue(inputChannel.isWaitingForFloatingBuffers()); @@ -490,7 +490,7 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { 14, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 14 buffers required in the channel", 14, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); // Recycle one floating buffer @@ -569,7 +569,7 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { 14, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 14 buffers required in the channel", 14, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); // Decrease the backlog to make the number of available buffers more than required buffers @@ -582,7 +582,7 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { 14, inputChannel.getNumberOfAvailableBuffers()); assertEquals("There should be 12 buffers required in the channel", 12, inputChannel.getNumberOfRequiredBuffers()); - assertEquals("There should be 0 buffer available in local pool", + assertEquals("There should be 0 buffers available in local pool", 0, bufferPool.getNumberOfAvailableMemorySegments()); // Recycle one exclusive buffer From 18ff2ce15bdb1e7bd246e438e47527a24559c86d Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Mon, 26 Feb 2018 17:50:10 +0100 Subject: [PATCH 024/268] [hotfix][network] minor improvements in UnionInputGate (cherry picked from commit 4203557) --- .../io/network/partition/consumer/UnionInputGate.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index 393e08775141a..44cdd526c5364 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -38,7 +38,7 @@ /** * Input gate wrapper to union the input from multiple input gates. * - *

    Each input gate has input channels attached from which it reads data. At each input gate, the + *

    Each input gate has input channels attached from which it reads data. At each input gate, the * input channels have unique IDs from 0 (inclusive) to the number of input channels (exclusive). * *

    @@ -49,7 +49,7 @@
      * +--------------+--------------+
      * 
    * - * The union input gate maps these IDs from 0 to the *total* number of input channels across all + *

    The union input gate maps these IDs from 0 to the *total* number of input channels across all * unioned input gates, e.g. the channels of input gate 0 keep their original indexes and the * channel indexes of input gate 1 are set off by 2 to 2--4. * @@ -183,11 +183,11 @@ public Optional getNextBufferOrEvent() throws IOException, Interr bufferOrEvent.setChannelIndex(channelIndexOffset + bufferOrEvent.getChannelIndex()); bufferOrEvent.setMoreAvailable(bufferOrEvent.moreAvailable() || inputGateWithData.moreInputGatesAvailable); - return Optional.ofNullable(bufferOrEvent); + return Optional.of(bufferOrEvent); } @Override - public Optional pollNextBufferOrEvent() throws IOException, InterruptedException { + public Optional pollNextBufferOrEvent() throws UnsupportedOperationException { throw new UnsupportedOperationException(); } @@ -217,7 +217,7 @@ private static class InputGateWithData { private final BufferOrEvent bufferOrEvent; private final boolean moreInputGatesAvailable; - public InputGateWithData(InputGate inputGate, BufferOrEvent bufferOrEvent, boolean moreInputGatesAvailable) { + InputGateWithData(InputGate inputGate, BufferOrEvent bufferOrEvent, boolean moreInputGatesAvailable) { this.inputGate = checkNotNull(inputGate); this.bufferOrEvent = checkNotNull(bufferOrEvent); this.moreInputGatesAvailable = moreInputGatesAvailable; From 9265666517830350a4a7037029e347f33df1bea2 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Mon, 26 Feb 2018 17:52:37 +0100 Subject: [PATCH 025/268] [FLINK-8737][network] disallow creating a union of UnionInputGate instances Recently, the pollNextBufferOrEvent() was added but not implemented but this is used in getNextBufferOrEvent() and thus any UnionInputGate containing a UnionInputGate would have failed already. There should be no use case for wiring up inputs this way. Therefore, fail early when trying to construct this. (cherry picked from commit e8de538) --- .../io/network/partition/consumer/UnionInputGate.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java index 44cdd526c5364..742592a93f0b3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnionInputGate.java @@ -61,7 +61,7 @@ * +--------------------+ * * - * It is possible to recursively union union input gates. + * It is NOT possible to recursively union union input gates. */ public class UnionInputGate implements InputGate, InputGateListener { @@ -103,6 +103,11 @@ public UnionInputGate(InputGate... inputGates) { int currentNumberOfInputChannels = 0; for (InputGate inputGate : inputGates) { + if (inputGate instanceof UnionInputGate) { + // if we want to add support for this, we need to implement pollNextBufferOrEvent() + throw new UnsupportedOperationException("Cannot union a union of input gates."); + } + // The offset to use for buffer or event instances received from this input gate. inputGateToIndexOffsetMap.put(checkNotNull(inputGate), currentNumberOfInputChannels); inputGatesWithRemainingData.add(inputGate); From 26c8f6c2a3ff75ffb954c816a57908318a2d8099 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Wed, 28 Feb 2018 12:15:30 +0100 Subject: [PATCH 026/268] [hotfix] [tests] Fix SelfConnectionITCase The test previously did not fail on failed execution, and thus evaluated incomplete results from a failed execution with th expected results. This cleans up serialization warnings and uses lambdas where possible, to make the code more readable. --- .../runtime/SelfConnectionITCase.java | 63 ++++--------------- 1 file changed, 12 insertions(+), 51 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java index b302513cd7867..a8023a01a594b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SelfConnectionITCase.java @@ -18,8 +18,6 @@ package org.apache.flink.test.streaming.runtime; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.co.CoMapFunction; @@ -38,6 +36,7 @@ /** * Integration tests for connected streams. */ +@SuppressWarnings("serial") public class SelfConnectionITCase extends AbstractTestBase { /** @@ -46,26 +45,17 @@ public class SelfConnectionITCase extends AbstractTestBase { @Test public void differentDataStreamSameChain() throws Exception { - TestListResultSink resultSink = new TestListResultSink(); + TestListResultSink resultSink = new TestListResultSink<>(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); DataStream src = env.fromElements(1, 3, 5); - DataStream stringMap = src.map(new MapFunction() { - private static final long serialVersionUID = 1L; - - @Override - public String map(Integer value) throws Exception { - return "x " + value; - } - }); + DataStream stringMap = src.map(value -> "x " + value); stringMap.connect(src).map(new CoMapFunction() { - private static final long serialVersionUID = 1L; - @Override public String map1(String value) { return value; @@ -94,55 +84,30 @@ public String map2(Integer value) { * (This is not actually self-connect.) */ @Test - public void differentDataStreamDifferentChain() { + public void differentDataStreamDifferentChain() throws Exception { - TestListResultSink resultSink = new TestListResultSink(); + TestListResultSink resultSink = new TestListResultSink<>(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(3); DataStream src = env.fromElements(1, 3, 5).disableChaining(); - DataStream stringMap = src.flatMap(new FlatMapFunction() { - - private static final long serialVersionUID = 1L; + DataStream stringMap = src + .flatMap(new FlatMapFunction() { @Override public void flatMap(Integer value, Collector out) throws Exception { out.collect("x " + value); } - }).keyBy(new KeySelector() { - - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(String value) throws Exception { - return value.length(); - } - }); - - DataStream longMap = src.map(new MapFunction() { + }).keyBy(String::length); - private static final long serialVersionUID = 1L; - - @Override - public Long map(Integer value) throws Exception { - return (long) (value + 1); - } - }).keyBy(new KeySelector() { - - private static final long serialVersionUID = 1L; - - @Override - public Integer getKey(Long value) throws Exception { - return value.intValue(); - } - }); + DataStream longMap = src + .map(value -> (long) (value + 1)) + .keyBy(Long::intValue); stringMap.connect(longMap).map(new CoMapFunction() { - private static final long serialVersionUID = 1L; - @Override public String map1(String value) { return value; @@ -154,11 +119,7 @@ public String map2(Long value) { } }).addSink(resultSink); - try { - env.execute(); - } catch (Exception e) { - e.printStackTrace(); - } + env.execute(); List expected = Arrays.asList("x 1", "x 3", "x 5", "2", "4", "6"); List result = resultSink.getResult(); From f60e46dafa8950d5e40cd8a3286c172ecaea6b73 Mon Sep 17 00:00:00 2001 From: gyao Date: Wed, 28 Feb 2018 13:04:19 +0100 Subject: [PATCH 027/268] [hotfix] Add missing space to log message in ZooKeeperLeaderElectionService --- .../runtime/leaderelection/ZooKeeperLeaderElectionService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java index 59d359218060c..dc0f3aefc6846 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionService.java @@ -199,7 +199,7 @@ public void confirmLeaderSessionID(UUID leaderSessionID) { } } } else { - LOG.warn("The leader session ID {} was confirmed even though the" + + LOG.warn("The leader session ID {} was confirmed even though the " + "corresponding JobManager was not elected as the leader.", leaderSessionID); } } From b7247929d0745b3b83306d0c93d97faf4ece4107 Mon Sep 17 00:00:00 2001 From: gyao Date: Wed, 28 Feb 2018 13:06:00 +0100 Subject: [PATCH 028/268] [hotfix][Javadoc] Fix typo in YARN Utils: teh -> the --- flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index 9ae5b543296b5..ff2478ede1415 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -320,7 +320,7 @@ public static Map getEnvironmentVariables(String envPrefix, org. * * @return The launch context for the TaskManager processes. * - * @throws Exception Thrown if teh launch context could not be created, for example if + * @throws Exception Thrown if the launch context could not be created, for example if * the resources could not be copied. */ static ContainerLaunchContext createTaskExecutorContext( From adb3750226971f7c67a0d3069103b56e4fee1c27 Mon Sep 17 00:00:00 2001 From: gyao Date: Wed, 28 Feb 2018 13:07:04 +0100 Subject: [PATCH 029/268] [hotfix][Javadoc] Fix typo in YarnTestBase: teh -> the --- .../src/test/java/org/apache/flink/yarn/YarnTestBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index c863e145f05e9..7bca32192489b 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -102,7 +102,7 @@ public abstract class YarnTestBase extends TestLogger { "Started SelectChannelConnector@0.0.0.0:8081" // Jetty should start on a random port in YARN mode. }; - /** These strings are white-listed, overriding teh prohibited strings. */ + /** These strings are white-listed, overriding the prohibited strings. */ protected static final String[] WHITELISTED_STRINGS = { "akka.remote.RemoteTransportExceptionNoStackTrace", // workaround for annoying InterruptedException logging: From 94bbd564ce5214b3366cc6d299fcb99ae62a2bd8 Mon Sep 17 00:00:00 2001 From: gyao Date: Wed, 28 Feb 2018 13:08:25 +0100 Subject: [PATCH 030/268] [hotfix][tests] Fix wrong assertEquals in YARNSessionCapacitySchedulerITCase Test swapped actual and expected arguments. Remove catching Throwable in test; instead propagate all exceptions. --- .../yarn/YARNSessionCapacitySchedulerITCase.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index 3a674ad082989..d00a9c447d135 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -416,7 +416,7 @@ public void perJobYarnClusterWithParallelism() throws IOException { * Test a fire-and-forget job submission to a YARN cluster. */ @Test(timeout = 60000) - public void testDetachedPerJobYarnCluster() throws IOException { + public void testDetachedPerJobYarnCluster() throws Exception { LOG.info("Starting testDetachedPerJobYarnCluster()"); File exampleJarLocation = new File("target/programs/BatchWordCount.jar"); @@ -432,7 +432,7 @@ public void testDetachedPerJobYarnCluster() throws IOException { * Test a fire-and-forget job submission to a YARN cluster. */ @Test(timeout = 60000) - public void testDetachedPerJobYarnClusterWithStreamingJob() throws IOException { + public void testDetachedPerJobYarnClusterWithStreamingJob() throws Exception { LOG.info("Starting testDetachedPerJobYarnClusterWithStreamingJob()"); File exampleJarLocation = new File("target/programs/StreamingWordCount.jar"); @@ -444,7 +444,7 @@ public void testDetachedPerJobYarnClusterWithStreamingJob() throws IOException { LOG.info("Finished testDetachedPerJobYarnClusterWithStreamingJob()"); } - private void testDetachedPerJobYarnClusterInternal(String job) throws IOException { + private void testDetachedPerJobYarnClusterInternal(String job) throws Exception { YarnClient yc = YarnClient.createYarnClient(); yc.init(YARN_CONFIGURATION); yc.start(); @@ -575,9 +575,6 @@ public boolean accept(File dir, String name) { } while (rep.getYarnApplicationState() == YarnApplicationState.RUNNING); verifyApplicationTags(rep); - } catch (Throwable t) { - LOG.warn("Error while detached yarn session was running", t); - Assert.fail(t.getMessage()); } finally { //cleanup the yarn-properties file @@ -625,7 +622,7 @@ private void verifyApplicationTags(final ApplicationReport report) throws Invoca @SuppressWarnings("unchecked") Set applicationTags = (Set) applicationTagsMethod.invoke(report); - Assert.assertEquals(applicationTags, Collections.singleton("test-tag")); + Assert.assertEquals(Collections.singleton("test-tag"), applicationTags); } @After From e92eb391cada27b49d32e7b11453fb5f06e19880 Mon Sep 17 00:00:00 2001 From: gyao Date: Wed, 28 Feb 2018 13:20:23 +0100 Subject: [PATCH 031/268] [FLINK-7805][flip6] Recover YARN containers after AM restart. Recover previously running containers after a restart of the ApplicationMaster. This is a port of a feature that was already implemented prior to FLIP-6. Extract RegisterApplicationMasterResponseReflector class into separate file. This closes #5597. --- ...terApplicationMasterResponseReflector.java | 102 +++++++++++++++ .../flink/yarn/YarnFlinkResourceManager.java | 52 -------- .../flink/yarn/YarnResourceManager.java | 17 ++- ...pplicationMasterResponseReflectorTest.java | 117 ++++++++++++++++++ 4 files changed, 235 insertions(+), 53 deletions(-) create mode 100644 flink-yarn/src/main/java/org/apache/flink/yarn/RegisterApplicationMasterResponseReflector.java create mode 100644 flink-yarn/src/test/java/org/apache/flink/yarn/RegisterApplicationMasterResponseReflectorTest.java diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/RegisterApplicationMasterResponseReflector.java b/flink-yarn/src/main/java/org/apache/flink/yarn/RegisterApplicationMasterResponseReflector.java new file mode 100644 index 0000000000000..13b5745d0cbc3 --- /dev/null +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/RegisterApplicationMasterResponseReflector.java @@ -0,0 +1,102 @@ +/* + * 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.yarn; + +import org.apache.flink.annotation.VisibleForTesting; + +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.Container; +import org.slf4j.Logger; + +import java.lang.reflect.Method; +import java.util.Collections; +import java.util.List; + +import static java.util.Objects.requireNonNull; + +/** + * Looks up the method {@link RegisterApplicationMasterResponse#getContainersFromPreviousAttempts()} + * once and saves the method. This saves computation time on subsequent calls. + */ +class RegisterApplicationMasterResponseReflector { + + private final Logger logger; + + /** + * Reflected method {@link RegisterApplicationMasterResponse#getContainersFromPreviousAttempts()}. + */ + private Method method; + + RegisterApplicationMasterResponseReflector(final Logger log) { + this(log, RegisterApplicationMasterResponse.class); + } + + @VisibleForTesting + RegisterApplicationMasterResponseReflector(final Logger log, final Class clazz) { + this.logger = requireNonNull(log); + requireNonNull(clazz); + + try { + method = clazz.getMethod("getContainersFromPreviousAttempts"); + } catch (NoSuchMethodException e) { + // that happens in earlier Hadoop versions (pre 2.2) + logger.info("Cannot reconnect to previously allocated containers. " + + "This YARN version does not support 'getContainersFromPreviousAttempts()'"); + } + } + + /** + * Checks if a YARN application still has registered containers. If the application master + * registered at the ResourceManager for the first time, this list will be empty. If the + * application master registered a repeated time (after a failure and recovery), this list + * will contain the containers that were previously allocated. + * + * @param response The response object from the registration at the ResourceManager. + * @return A list with containers from previous application attempt. + */ + List getContainersFromPreviousAttempts(final RegisterApplicationMasterResponse response) { + return getContainersFromPreviousAttemptsUnsafe(response); + } + + /** + * Same as {@link #getContainersFromPreviousAttempts(RegisterApplicationMasterResponse)} but + * allows to pass objects that are not of type {@link RegisterApplicationMasterResponse}. + */ + @VisibleForTesting + List getContainersFromPreviousAttemptsUnsafe(final Object response) { + if (method != null && response != null) { + try { + @SuppressWarnings("unchecked") + final List containers = (List) method.invoke(response); + if (containers != null && !containers.isEmpty()) { + return containers; + } + } catch (Exception t) { + logger.error("Error invoking 'getContainersFromPreviousAttempts()'", t); + } + } + + return Collections.emptyList(); + } + + @VisibleForTesting + Method getMethod() { + return method; + } +} diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java index 4d8142f7b830a..8e686bbbe34c1 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java @@ -47,10 +47,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.slf4j.Logger; -import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -616,56 +614,6 @@ private FinalApplicationStatus getYarnStatus(ApplicationStatus status) { } } - /** - * Looks up the getContainersFromPreviousAttempts method on RegisterApplicationMasterResponse - * once and saves the method. This saves computation time on the sequent calls. - */ - private static class RegisterApplicationMasterResponseReflector { - - private Logger logger; - private Method method; - - public RegisterApplicationMasterResponseReflector(Logger log) { - this.logger = log; - - try { - method = RegisterApplicationMasterResponse.class - .getMethod("getContainersFromPreviousAttempts"); - - } catch (NoSuchMethodException e) { - // that happens in earlier Hadoop versions - logger.info("Cannot reconnect to previously allocated containers. " + - "This YARN version does not support 'getContainersFromPreviousAttempts()'"); - } - } - - /** - * Checks if a YARN application still has registered containers. If the application master - * registered at the ResourceManager for the first time, this list will be empty. If the - * application master registered a repeated time (after a failure and recovery), this list - * will contain the containers that were previously allocated. - * - * @param response The response object from the registration at the ResourceManager. - * @return A list with containers from previous application attempt. - */ - private List getContainersFromPreviousAttempts(RegisterApplicationMasterResponse response) { - if (method != null && response != null) { - try { - @SuppressWarnings("unchecked") - List list = (List) method.invoke(response); - if (list != null && !list.isEmpty()) { - return list; - } - } catch (Throwable t) { - logger.error("Error invoking 'getContainersFromPreviousAttempts()'", t); - } - } - - return Collections.emptyList(); - } - - } - // ------------------------------------------------------------------------ // Actor props factory // ------------------------------------------------------------------------ diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 5380356de7965..f3ec04bf1e6e0 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -41,6 +41,7 @@ import org.apache.flink.yarn.configuration.YarnConfigOptions; import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerStatus; @@ -192,11 +193,24 @@ protected AMRMClientAsync createAndStartResourceMan restPort = -1; } - resourceManagerClient.registerApplicationMaster(hostPort.f0, restPort, webInterfaceUrl); + final RegisterApplicationMasterResponse registerApplicationMasterResponse = + resourceManagerClient.registerApplicationMaster(hostPort.f0, restPort, webInterfaceUrl); + getContainersFromPreviousAttempts(registerApplicationMasterResponse); return resourceManagerClient; } + private void getContainersFromPreviousAttempts(final RegisterApplicationMasterResponse registerApplicationMasterResponse) { + final List containersFromPreviousAttempts = + new RegisterApplicationMasterResponseReflector(log).getContainersFromPreviousAttempts(registerApplicationMasterResponse); + + log.info("Recovered {} containers from previous attempts ({}).", containersFromPreviousAttempts.size(), containersFromPreviousAttempts); + + for (final Container container : containersFromPreviousAttempts) { + workerNodeMap.put(new ResourceID(container.getId().toString()), new YarnWorkerNode(container)); + } + } + protected NMClient createAndStartNodeManagerClient(YarnConfiguration yarnConfiguration) { // create the client to communicate with the node managers NMClient nodeManagerClient = NMClient.createNMClient(); @@ -315,6 +329,7 @@ public void onContainersCompleted(List list) { closeTaskManagerConnection(new ResourceID( container.getContainerId().toString()), new Exception(container.getDiagnostics())); } + workerNodeMap.remove(new ResourceID(container.getContainerId().toString())); } } diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/RegisterApplicationMasterResponseReflectorTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/RegisterApplicationMasterResponseReflectorTest.java new file mode 100644 index 0000000000000..af33e65c27a2e --- /dev/null +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/RegisterApplicationMasterResponseReflectorTest.java @@ -0,0 +1,117 @@ +/* + * 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.yarn; + +import org.apache.flink.util.TestLogger; + +import org.apache.hadoop.util.VersionInfo; +import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse; +import org.apache.hadoop.yarn.api.records.Container; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Method; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assert.assertThat; +import static org.junit.Assume.assumeTrue; + +/** + * Tests for {@link RegisterApplicationMasterResponseReflector}. + */ +public class RegisterApplicationMasterResponseReflectorTest extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(RegisterApplicationMasterResponseReflectorTest.class); + + @Mock + private Container mockContainer; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testCallsMethodIfPresent() { + final RegisterApplicationMasterResponseReflector registerApplicationMasterResponseReflector = + new RegisterApplicationMasterResponseReflector(LOG, HasMethod.class); + + final List containersFromPreviousAttemptsUnsafe = + registerApplicationMasterResponseReflector.getContainersFromPreviousAttemptsUnsafe(new + HasMethod()); + + assertThat(containersFromPreviousAttemptsUnsafe, hasSize(1)); + } + + @Test + public void testDoesntCallMethodIfAbsent() { + final RegisterApplicationMasterResponseReflector registerApplicationMasterResponseReflector = + new RegisterApplicationMasterResponseReflector(LOG, HasMethod.class); + + final List containersFromPreviousAttemptsUnsafe = + registerApplicationMasterResponseReflector.getContainersFromPreviousAttemptsUnsafe(new + Object()); + + assertThat(containersFromPreviousAttemptsUnsafe, empty()); + } + + @Test + public void testGetMethodReflectiveHadoop22() { + assumeTrue( + "Method getContainersFromPreviousAttempts is not supported by Hadoop: " + + VersionInfo.getVersion(), + isHadoopVersionGreaterThanOrEquals(2, 2)); + + final RegisterApplicationMasterResponseReflector registerApplicationMasterResponseReflector = + new RegisterApplicationMasterResponseReflector(LOG); + + final Method method = registerApplicationMasterResponseReflector.getMethod(); + assertThat(method, notNullValue()); + } + + private static boolean isHadoopVersionGreaterThanOrEquals(final int major, final int minor) { + final String[] splitVersion = VersionInfo.getVersion().split("\\."); + final int[] versions = Arrays.stream(splitVersion).mapToInt(Integer::parseInt).toArray(); + return versions[0] >= major && versions[1] >= minor; + } + + /** + * Class which has a method with the same signature as + * {@link RegisterApplicationMasterResponse#getContainersFromPreviousAttempts()}. + */ + private class HasMethod { + + /** + * Called from {@link #testCallsMethodIfPresent()}. + */ + @SuppressWarnings("unused") + public List getContainersFromPreviousAttempts() { + return Collections.singletonList(mockContainer); + } + } +} From cc32ffe70b70b926168ea91a7e92f90187dbbe57 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 1 Mar 2018 09:43:56 +0100 Subject: [PATCH 032/268] [FLINK-4387][QS] don't wait and process requests at Netty servers after shutdown request There is a race condition on an assertion in Netty's event loop that may cause tests to fail when finished early. This was fixed in 4.0.33.Final, see https://github.com/netty/netty/issues/4357. --- .../org/apache/flink/queryablestate/network/ClientTest.java | 3 ++- .../flink/queryablestate/network/KvStateServerTest.java | 6 ++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java index 8638efa680fc0..6aa4710942e6d 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/ClientTest.java @@ -111,7 +111,8 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { if (nioGroup != null) { - nioGroup.shutdownGracefully(); + // note: no "quiet period" to not trigger Netty#4357 + nioGroup.shutdownGracefully(0, 10, TimeUnit.SECONDS); } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java index 8af9cf58851fa..79c23ad2a2d95 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/network/KvStateServerTest.java @@ -79,7 +79,8 @@ public class KvStateServerTest { @AfterClass public static void tearDown() throws Exception { if (NIO_GROUP != null) { - NIO_GROUP.shutdownGracefully(); + // note: no "quiet period" to not trigger Netty#4357 + NIO_GROUP.shutdownGracefully(0, 10, TimeUnit.SECONDS); } } @@ -191,7 +192,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception if (bootstrap != null) { EventLoopGroup group = bootstrap.group(); if (group != null) { - group.shutdownGracefully(); + // note: no "quiet period" to not trigger Netty#4357 + group.shutdownGracefully(0, 10, TimeUnit.SECONDS); } } } From 4b8c2a43cb79e8a1b9ec4cae623cbeb19925704e Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 1 Mar 2018 10:58:14 +0100 Subject: [PATCH 033/268] [hotfix] [formats] Make ObjectMapper final in JsonNodeDeserializationSchema --- .../flink/formats/json/JsonNodeDeserializationSchema.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java index 7501cc3a83619..e20c83977343a 100644 --- a/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java +++ b/flink-formats/flink-json/src/main/java/org/apache/flink/formats/json/JsonNodeDeserializationSchema.java @@ -35,13 +35,10 @@ public class JsonNodeDeserializationSchema extends AbstractDeserializationSchema private static final long serialVersionUID = -1699854177598621044L; - private ObjectMapper mapper; + private final ObjectMapper mapper = new ObjectMapper(); @Override public ObjectNode deserialize(byte[] message) throws IOException { - if (mapper == null) { - mapper = new ObjectMapper(); - } return mapper.readValue(message, ObjectNode.class); } } From 9c0e75f8ddbbfb247d4e538a00eeb5c6586e9d39 Mon Sep 17 00:00:00 2001 From: zentol Date: Thu, 1 Mar 2018 11:52:28 +0100 Subject: [PATCH 034/268] [hotfix][build] Add missing shade-plugin execution id's 2 metric modules weren't setting the execution id to "shade-flink" causing them to not pick up the default shade-plugin configuration. --- flink-metrics/flink-metrics-datadog/pom.xml | 1 + flink-metrics/flink-metrics-prometheus/pom.xml | 1 + 2 files changed, 2 insertions(+) diff --git a/flink-metrics/flink-metrics-datadog/pom.xml b/flink-metrics/flink-metrics-datadog/pom.xml index f62afd1f36067..b466f4324fe50 100644 --- a/flink-metrics/flink-metrics-datadog/pom.xml +++ b/flink-metrics/flink-metrics-datadog/pom.xml @@ -60,6 +60,7 @@ under the License. maven-shade-plugin + shade-flink package shade diff --git a/flink-metrics/flink-metrics-prometheus/pom.xml b/flink-metrics/flink-metrics-prometheus/pom.xml index 544843c3cf567..56fc99944159d 100644 --- a/flink-metrics/flink-metrics-prometheus/pom.xml +++ b/flink-metrics/flink-metrics-prometheus/pom.xml @@ -105,6 +105,7 @@ under the License. maven-shade-plugin + shade-flink package shade From 06b05cd204bd9a12884ad12805a61005ef40fbe7 Mon Sep 17 00:00:00 2001 From: Jelmer Kuperus Date: Wed, 28 Feb 2018 21:34:08 +0100 Subject: [PATCH 035/268] [FLINK-8814] [file system sinks] Control over the extension of part files created by BucketingSink. --- .../fs/bucketing/BucketingSink.java | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java index 6e7f460a6dc63..faf3c566803e8 100644 --- a/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java +++ b/flink-connectors/flink-connector-filesystem/src/main/java/org/apache/flink/streaming/connectors/fs/bucketing/BucketingSink.java @@ -226,7 +226,12 @@ public class BucketingSink /** * The default prefix for part files. */ - private static final String DEFAULT_PART_REFIX = "part"; + private static final String DEFAULT_PART_PREFIX = "part"; + + /** + * The default suffix for part files. + */ + private static final String DEFAULT_PART_SUFFIX = null; /** * The default timeout for asynchronous operations such as recoverLease and truncate (in {@code ms}). @@ -263,7 +268,8 @@ public class BucketingSink private String validLengthSuffix = DEFAULT_VALID_SUFFIX; private String validLengthPrefix = DEFAULT_VALID_PREFIX; - private String partPrefix = DEFAULT_PART_REFIX; + private String partPrefix = DEFAULT_PART_PREFIX; + private String partSuffix = DEFAULT_PART_SUFFIX; private boolean useTruncate = true; @@ -530,6 +536,10 @@ private void openNewPartFile(Path bucketPath, BucketState bucketState) throws partPath = new Path(bucketPath, partPrefix + "-" + subtaskIndex + "-" + bucketState.partCounter); } + if (partSuffix != null) { + partPath = partPath.suffix(partSuffix); + } + // increase, so we don't have to check for this name next time bucketState.partCounter++; @@ -986,6 +996,14 @@ public BucketingSink setValidLengthPrefix(String validLengthPrefix) { return this; } + /** + * Sets the prefix of part files. The default is no suffix. + */ + public BucketingSink setPartSuffix(String partSuffix) { + this.partSuffix = partSuffix; + return this; + } + /** * Sets the prefix of part files. The default is {@code "part"}. */ From 122e2eb4606afd0cdf20ab3bf82bb524bb9c3673 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 28 Feb 2018 18:02:40 +0100 Subject: [PATCH 036/268] [FLINK-8810] Move end-to-end test scripts to end-to-end module This also makes the tests executable by calling $ flink-end-to-end-tests/run-pre-commit-tests.sh --- flink-end-to-end-tests/README.md | 24 +++++ .../run-pre-commit-tests.sh | 98 +++++++++++++++++++ .../test-scripts}/common.sh | 0 .../test-scripts}/test-data/words | 0 .../test-scripts}/test_batch_wordcount.sh | 0 .../test-scripts}/test_hadoop_free.sh | 0 .../test-scripts}/test_shaded_hadoop_s3a.sh | 0 .../test-scripts}/test_shaded_presto_s3.sh | 0 .../test_streaming_classloader.sh | 0 .../test-scripts}/test_streaming_kafka010.sh | 0 .../test_streaming_python_wordcount.sh | 0 pom.xml | 2 +- tools/travis_mvn_watchdog.sh | 58 +---------- 13 files changed, 126 insertions(+), 56 deletions(-) create mode 100644 flink-end-to-end-tests/README.md create mode 100755 flink-end-to-end-tests/run-pre-commit-tests.sh rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/common.sh (100%) rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/test-data/words (100%) rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/test_batch_wordcount.sh (100%) rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/test_hadoop_free.sh (100%) rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/test_shaded_hadoop_s3a.sh (100%) rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/test_shaded_presto_s3.sh (100%) rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/test_streaming_classloader.sh (100%) rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/test_streaming_kafka010.sh (100%) rename {test-infra/end-to-end-test => flink-end-to-end-tests/test-scripts}/test_streaming_python_wordcount.sh (100%) diff --git a/flink-end-to-end-tests/README.md b/flink-end-to-end-tests/README.md new file mode 100644 index 0000000000000..1c8aadcc6177b --- /dev/null +++ b/flink-end-to-end-tests/README.md @@ -0,0 +1,24 @@ +# Flink End-to-End Tests + +This module contains tests that verify end-to-end behaviour of Flink. + +## Running Tests +You can run all tests by executing + +``` +$ FLINK_DIR= flink-end-to-end-tests/run-pre-commit-tests.sh +``` + +where is a Flink distribution directory. + +You can also run tests individually via + +``` +$ FLINK_DIR= flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh +``` + +## Writing Tests + +Have a look at test_batch_wordcount.sh for a very basic test and +test_streaming_kafka010.sh for a more involved example. Whenever possible, try +to put new functionality in common.sh so that it can be reused by other tests. diff --git a/flink-end-to-end-tests/run-pre-commit-tests.sh b/flink-end-to-end-tests/run-pre-commit-tests.sh new file mode 100755 index 0000000000000..2c1810b91c8bf --- /dev/null +++ b/flink-end-to-end-tests/run-pre-commit-tests.sh @@ -0,0 +1,98 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd )`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd )`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +EXIT_CODE=0 + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Wordcount end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_batch_wordcount.sh + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Kafka end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_streaming_kafka010.sh + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running class loading end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_streaming_classloader.sh + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Shaded Hadoop S3A end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_shaded_hadoop_s3a.sh + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Shaded Presto S3 end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_shaded_presto_s3.sh + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Hadoop-free Wordcount end-to-end test\n" + printf "==============================================================================\n" + CLUSTER_MODE=cluster $END_TO_END_DIR/test-scripts/test_hadoop_free.sh + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Streaming Python Wordcount end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_streaming_python_wordcount.sh + EXIT_CODE=$? +fi + + +# Exit code for Travis build success/failure +exit $EXIT_CODE diff --git a/test-infra/end-to-end-test/common.sh b/flink-end-to-end-tests/test-scripts/common.sh similarity index 100% rename from test-infra/end-to-end-test/common.sh rename to flink-end-to-end-tests/test-scripts/common.sh diff --git a/test-infra/end-to-end-test/test-data/words b/flink-end-to-end-tests/test-scripts/test-data/words similarity index 100% rename from test-infra/end-to-end-test/test-data/words rename to flink-end-to-end-tests/test-scripts/test-data/words diff --git a/test-infra/end-to-end-test/test_batch_wordcount.sh b/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh similarity index 100% rename from test-infra/end-to-end-test/test_batch_wordcount.sh rename to flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh diff --git a/test-infra/end-to-end-test/test_hadoop_free.sh b/flink-end-to-end-tests/test-scripts/test_hadoop_free.sh similarity index 100% rename from test-infra/end-to-end-test/test_hadoop_free.sh rename to flink-end-to-end-tests/test-scripts/test_hadoop_free.sh diff --git a/test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh b/flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh similarity index 100% rename from test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh rename to flink-end-to-end-tests/test-scripts/test_shaded_hadoop_s3a.sh diff --git a/test-infra/end-to-end-test/test_shaded_presto_s3.sh b/flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh similarity index 100% rename from test-infra/end-to-end-test/test_shaded_presto_s3.sh rename to flink-end-to-end-tests/test-scripts/test_shaded_presto_s3.sh diff --git a/test-infra/end-to-end-test/test_streaming_classloader.sh b/flink-end-to-end-tests/test-scripts/test_streaming_classloader.sh similarity index 100% rename from test-infra/end-to-end-test/test_streaming_classloader.sh rename to flink-end-to-end-tests/test-scripts/test_streaming_classloader.sh diff --git a/test-infra/end-to-end-test/test_streaming_kafka010.sh b/flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh similarity index 100% rename from test-infra/end-to-end-test/test_streaming_kafka010.sh rename to flink-end-to-end-tests/test-scripts/test_streaming_kafka010.sh diff --git a/test-infra/end-to-end-test/test_streaming_python_wordcount.sh b/flink-end-to-end-tests/test-scripts/test_streaming_python_wordcount.sh similarity index 100% rename from test-infra/end-to-end-test/test_streaming_python_wordcount.sh rename to flink-end-to-end-tests/test-scripts/test_streaming_python_wordcount.sh diff --git a/pom.xml b/pom.xml index 5f0ebb4a00c95..dd1fa3729aa6f 100644 --- a/pom.xml +++ b/pom.xml @@ -1017,7 +1017,7 @@ under the License. out/test/flink-avro/avro/user.avsc flink-libraries/flink-table/src/test/scala/resources/*.out flink-yarn/src/test/resources/krb5.keytab - test-infra/end-to-end-test/test-data/* + flink-end-to-end-tests/test-scripts/test-data/* **/src/test/resources/*-snapshot diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index 4b1c2e31ad5da..dc1125d529685 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -576,61 +576,9 @@ case $TEST in printf "Running end-to-end tests\n" printf "==============================================================================\n" - if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Running Wordcount end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target test-infra/end-to-end-test/test_batch_wordcount.sh - EXIT_CODE=$? - fi - - if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Running Kafka end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target test-infra/end-to-end-test/test_streaming_kafka010.sh - EXIT_CODE=$? - fi - - if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Running class loading end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target test-infra/end-to-end-test/test_streaming_classloader.sh - EXIT_CODE=$? - fi - - if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Running Shaded Hadoop S3A end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target test-infra/end-to-end-test/test_shaded_hadoop_s3a.sh - EXIT_CODE=$? - fi - - if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Running Shaded Presto S3 end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target test-infra/end-to-end-test/test_shaded_presto_s3.sh - EXIT_CODE=$? - fi - - if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Running Hadoop-free Wordcount end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target CLUSTER_MODE=cluster test-infra/end-to-end-test/test_hadoop_free.sh - EXIT_CODE=$? - fi - - if [ $EXIT_CODE == 0 ]; then - printf "\n==============================================================================\n" - printf "Running Streaming Python Wordcount end-to-end test\n" - printf "==============================================================================\n" - FLINK_DIR=build-target test-infra/end-to-end-test/test_streaming_python_wordcount.sh - EXIT_CODE=$? - fi + FLINK_DIR=build-target flink-end-to-end-tests/run-pre-commit-tests.sh + + EXIT_CODE=$? else printf "\n==============================================================================\n" printf "Previous build failure detected, skipping end-to-end tests.\n" From 155dd0db6bd70615ac168b4f8ec13abe6aa2b66c Mon Sep 17 00:00:00 2001 From: zjureel Date: Thu, 21 Dec 2017 17:49:11 +0800 Subject: [PATCH 037/268] [FLINK-6352] [kafka] Support to set offset of Kafka with specific date --- .../kafka/FlinkKafkaConsumer010.java | 33 +- .../kafka/internal/Kafka010Fetcher.java | 9 +- .../internal/KafkaConsumerCallBridge010.java | 25 ++ .../connectors/kafka/Kafka010ITCase.java | 35 +- .../kafka/internal/Kafka010FetcherTest.java | 9 +- .../connectors/kafka/Kafka011ITCase.java | 34 ++ .../connectors/kafka/Kafka08ITCase.java | 4 +- .../kafka/FlinkKafkaConsumer09.java | 3 +- .../kafka/internal/Kafka09Fetcher.java | 8 +- .../internal/KafkaConsumerCallBridge.java | 3 + .../kafka/internal/KafkaConsumerThread.java | 3 + .../kafka/internal/Kafka09FetcherTest.java | 9 +- .../kafka/FlinkKafkaConsumerBase.java | 4 +- .../connectors/kafka/config/StartupMode.java | 3 + .../KafkaTopicPartitionStateSentinel.java | 3 + .../kafka/KafkaConsumerTestBase.java | 317 ++++++++++++++++-- 16 files changed, 466 insertions(+), 36 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java index 394d1239a92f8..d5bd1653dc3f5 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java @@ -24,6 +24,7 @@ import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher; import org.apache.flink.streaming.connectors.kafka.internal.Kafka010PartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; @@ -37,11 +38,14 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import java.util.Collections; +import java.util.Date; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; +import static org.apache.flink.util.Preconditions.checkArgument; + /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from * Apache Kafka 0.10.x. The consumer can run in multiple parallel instances, each of which will pull @@ -64,6 +68,8 @@ public class FlinkKafkaConsumer010 extends FlinkKafkaConsumer09 { private static final long serialVersionUID = 2324564345203409112L; + private Date specificStartupDate = null; + // ------------------------------------------------------------------------ /** @@ -172,6 +178,30 @@ public FlinkKafkaConsumer010(Pattern subscriptionPattern, KeyedDeserializationSc super(subscriptionPattern, deserializer, props); } + + /** + * Specifies the consumer to start reading partitions from specific date. The specified date must before current timestamp. + * This lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. + * + *

    The consumer will look up the earliest offset whose timestamp is greater than or equal to the specific date from Kafka. + * If there's no such offset, the consumer will use the latest offset to read data from kafka. + * + *

    This method does not effect where partitions are read from when the consumer is restored + * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + * savepoint, only the offsets in the restored state will be used. + * + * @return The consumer object, to allow function chaining. + */ + public FlinkKafkaConsumer010 setStartFromSpecificDate(Date date) { + Date now = new Date(); + checkArgument(null != date && date.getTime() <= now.getTime(), + "Startup time[" + date + "] must be before current time[" + now + "]."); + this.startupMode = StartupMode.SPECIFIC_TIMESTAMP; + this.specificStartupDate = date; + this.specificStartupOffsets = null; + return this; + } + @Override protected AbstractFetcher createFetcher( SourceContext sourceContext, @@ -203,7 +233,8 @@ public FlinkKafkaConsumer010(Pattern subscriptionPattern, KeyedDeserializationSc pollTimeout, runtimeContext.getMetricGroup(), consumerMetricGroup, - useMetrics); + useMetrics, + specificStartupDate); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java index 9b9b217b205f0..9ed15fc272a59 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java @@ -32,6 +32,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import java.util.Date; import java.util.Map; import java.util.Properties; @@ -60,7 +61,8 @@ public Kafka010Fetcher( long pollTimeout, MetricGroup subtaskMetricGroup, MetricGroup consumerMetricGroup, - boolean useMetrics) throws Exception { + boolean useMetrics, + Date startupDate) throws Exception { super( sourceContext, assignedPartitionsWithInitialOffsets, @@ -75,7 +77,8 @@ public Kafka010Fetcher( pollTimeout, subtaskMetricGroup, consumerMetricGroup, - useMetrics); + useMetrics, + startupDate); } @Override @@ -95,7 +98,7 @@ protected void emitRecord( */ @Override protected KafkaConsumerCallBridge010 createCallBridge() { - return new KafkaConsumerCallBridge010(); + return new KafkaConsumerCallBridge010(startupDate); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java index 5815bfade38de..2f430c5a1a7b1 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java @@ -21,10 +21,14 @@ import org.apache.flink.annotation.Internal; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; import java.util.Collections; +import java.util.Date; +import java.util.HashMap; import java.util.List; +import java.util.Map; /** * The ConsumerCallBridge simply calls the {@link KafkaConsumer#assign(java.util.Collection)} method. @@ -37,6 +41,13 @@ @Internal public class KafkaConsumerCallBridge010 extends KafkaConsumerCallBridge { + private Date startupDate; + + public KafkaConsumerCallBridge010(Date startupDate) { + super(); + this.startupDate = startupDate; + } + @Override public void assignPartitions(KafkaConsumer consumer, List topicPartitions) throws Exception { consumer.assign(topicPartitions); @@ -51,4 +62,18 @@ public void seekPartitionToBeginning(KafkaConsumer consumer, TopicPartitio public void seekPartitionToEnd(KafkaConsumer consumer, TopicPartition partition) { consumer.seekToEnd(Collections.singletonList(partition)); } + + @Override + public void seekPartitionToDate(KafkaConsumer consumer, TopicPartition partition) { + Map partitionTimestampMap = new HashMap<>(1); + partitionTimestampMap.put(partition, startupDate.getTime()); + + Map topicPartitionOffsetMap = consumer.offsetsForTimes(partitionTimestampMap); + OffsetAndTimestamp offsetAndTimestamp = null == topicPartitionOffsetMap ? null : topicPartitionOffsetMap.get(partition); + if (null == offsetAndTimestamp) { + seekPartitionToEnd(consumer, partition); + } else { + consumer.seek(partition, offsetAndTimestamp.offset()); + } + } } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java index c2b3dfa1056bd..f821bf66e193b 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.core.memory.DataInputView; @@ -34,6 +35,8 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; @@ -45,6 +48,8 @@ import java.io.ByteArrayInputStream; import java.io.IOException; +import java.util.Date; +import java.util.Map; /** * IT cases for Kafka 0.10 . @@ -155,6 +160,11 @@ public void testStartFromSpecificOffsets() throws Exception { runStartFromSpecificOffsets(); } + @Test(timeout = 60000) + public void testStartFromSpecificDate() throws Exception { + runStartFromSpecificDate(); + } + // --- offset committing --- @Test(timeout = 60000) @@ -254,6 +264,30 @@ public long extractTimestamp(Long element, long previousElementTimestamp) { deleteTestTopic(topic); } + @Override + protected void setKafkaConsumerOffset(final StartupMode startupMode, + final FlinkKafkaConsumerBase> consumer, + final Map specificStartupOffsets, + final Date specificStartupDate) { + switch (startupMode) { + case EARLIEST: + consumer.setStartFromEarliest(); + break; + case LATEST: + consumer.setStartFromLatest(); + break; + case SPECIFIC_OFFSETS: + consumer.setStartFromSpecificOffsets(specificStartupOffsets); + break; + case GROUP_OFFSETS: + consumer.setStartFromGroupOffsets(); + break; + case SPECIFIC_TIMESTAMP: + ((FlinkKafkaConsumer010>) consumer).setStartFromSpecificDate(specificStartupDate); + break; + } + } + private static class TimestampValidatingOperator extends StreamSink { private static final long serialVersionUID = 1353168781235526806L; @@ -339,5 +373,4 @@ public boolean isEndOfStream(Long nextElement) { return cnt > 1000L; } } - } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java index f57fbea67e147..accf487dc439e 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java @@ -131,7 +131,8 @@ public Void answer(InvocationOnMock invocation) { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false); + false, + null); // ----- run the fetcher ----- @@ -268,7 +269,8 @@ public Void answer(InvocationOnMock invocation) { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false); + false, + null); // ----- run the fetcher ----- @@ -383,7 +385,8 @@ public void testCancellationWhenEmitBlocks() throws Exception { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false); + false, + null); // ----- run the fetcher ----- diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java index 99d5b56d94bc1..1bbbdb4ab5a6a 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.core.memory.DataInputView; @@ -34,6 +35,8 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; @@ -46,6 +49,8 @@ import java.io.ByteArrayInputStream; import java.io.IOException; +import java.util.Date; +import java.util.Map; import java.util.Optional; /** @@ -163,6 +168,11 @@ public void testStartFromSpecificOffsets() throws Exception { runStartFromSpecificOffsets(); } + @Test(timeout = 60000) + public void testStartFromSpecificDate() throws Exception { + runStartFromSpecificDate(); + } + // --- offset committing --- @Test(timeout = 60000) @@ -264,6 +274,30 @@ public long extractTimestamp(Long element, long previousElementTimestamp) { deleteTestTopic(topic); } + @Override + protected void setKafkaConsumerOffset(final StartupMode startupMode, + final FlinkKafkaConsumerBase> consumer, + final Map specificStartupOffsets, + final Date specificStartupDate) { + switch (startupMode) { + case EARLIEST: + consumer.setStartFromEarliest(); + break; + case LATEST: + consumer.setStartFromLatest(); + break; + case SPECIFIC_OFFSETS: + consumer.setStartFromSpecificOffsets(specificStartupOffsets); + break; + case GROUP_OFFSETS: + consumer.setStartFromGroupOffsets(); + break; + case SPECIFIC_TIMESTAMP: + ((FlinkKafkaConsumer011>) consumer).setStartFromSpecificDate(specificStartupDate); + break; + } + } + private static class TimestampValidatingOperator extends StreamSink { private static final long serialVersionUID = 1353168781235526806L; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java index b3afa579aab6e..6abccde9b4b83 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08ITCase.java @@ -99,7 +99,7 @@ public void testInvalidOffset() throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.getConfig().disableSysoutLogging(); - readSequence(env, StartupMode.GROUP_OFFSETS, null, standardProps, parallelism, topic, valuesCount, startFrom); + readSequence(env, StartupMode.GROUP_OFFSETS, null, null, standardProps, parallelism, topic, valuesCount, startFrom); deleteTestTopic(topic); } @@ -212,7 +212,7 @@ public void testOffsetAutocommitTest() throws Exception { readProps.setProperty("auto.commit.interval.ms", "500"); // read so that the offset can be committed to ZK - readSequence(env, StartupMode.GROUP_OFFSETS, null, readProps, parallelism, topicName, 100, 0); + readSequence(env, StartupMode.GROUP_OFFSETS, null, null, readProps, parallelism, topicName, 100, 0); // get the offset CuratorFramework curatorFramework = ((KafkaTestEnvironmentImpl) kafkaServer).createCuratorClient(); diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java index 6e6051b35e40d..4c3fa32cd1111 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java @@ -260,7 +260,8 @@ private FlinkKafkaConsumer09( pollTimeout, runtimeContext.getMetricGroup(), consumerMetricGroup, - useMetrics); + useMetrics, + null); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index dcc67d5b4fb14..185760bd680ad 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -40,6 +40,7 @@ import javax.annotation.Nonnull; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -71,6 +72,9 @@ public class Kafka09Fetcher extends AbstractFetcher { /** Flag to mark the main work loop as alive. */ private volatile boolean running = true; + /** The specific date used to set the offset of kafka. */ + protected Date startupDate; + // ------------------------------------------------------------------------ public Kafka09Fetcher( @@ -87,7 +91,8 @@ public Kafka09Fetcher( long pollTimeout, MetricGroup subtaskMetricGroup, MetricGroup consumerMetricGroup, - boolean useMetrics) throws Exception { + boolean useMetrics, + Date startupDate) throws Exception { super( sourceContext, assignedPartitionsWithInitialOffsets, @@ -101,6 +106,7 @@ public Kafka09Fetcher( this.deserializer = deserializer; this.handover = new Handover(); + this.startupDate = startupDate; this.consumerThread = new KafkaConsumerThread( LOG, diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java index b789633333df2..ac8ef7a0c93f5 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java @@ -50,4 +50,7 @@ public void seekPartitionToEnd(KafkaConsumer consumer, TopicPartition part consumer.seekToEnd(partition); } + public void seekPartitionToDate(KafkaConsumer consumer, TopicPartition partition) { + throw new RuntimeException("Seek offset from a specific date is only supported for version 0.10 and later."); + } } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java index 38e8a41d474cf..6f8c699f4fd8c 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java @@ -423,6 +423,9 @@ void reassignPartitions(List> newPartit } else if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.LATEST_OFFSET) { consumerCallBridge.seekPartitionToEnd(consumerTmp, newPartitionState.getKafkaPartitionHandle()); newPartitionState.setOffset(consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1); + } else if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.TIMESTAMP) { + consumerCallBridge.seekPartitionToDate(consumerTmp, newPartitionState.getKafkaPartitionHandle()); + newPartitionState.setOffset(consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1); } else if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.GROUP_OFFSET) { // the KafkaConsumer by default will automatically seek the consumer position // to the committed group offset, so we do not need to do it. diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java index 27b67f1098413..749c2dbdc4132 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java @@ -131,7 +131,8 @@ public Void answer(InvocationOnMock invocation) { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false); + false, + null); // ----- run the fetcher ----- @@ -267,7 +268,8 @@ public Void answer(InvocationOnMock invocation) { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false); + false, + null); // ----- run the fetcher ----- @@ -382,7 +384,8 @@ public void testCancellationWhenEmitBlocks() throws Exception { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false); + false, + null); // ----- run the fetcher ----- diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index df35de61f03c5..94230172deec1 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -147,10 +147,10 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti private final long discoveryIntervalMillis; /** The startup mode for the consumer (default is {@link StartupMode#GROUP_OFFSETS}). */ - private StartupMode startupMode = StartupMode.GROUP_OFFSETS; + protected StartupMode startupMode = StartupMode.GROUP_OFFSETS; /** Specific startup offsets; only relevant when startup mode is {@link StartupMode#SPECIFIC_OFFSETS}. */ - private Map specificStartupOffsets; + protected Map specificStartupOffsets; // ------------------------------------------------------------------------ // runtime state (used individually by each parallel subtask) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java index f984c8254abd6..ec0a7e8dd0e30 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java @@ -35,6 +35,9 @@ public enum StartupMode { /** Start from the latest offset. */ LATEST(KafkaTopicPartitionStateSentinel.LATEST_OFFSET), + /** Start from specific timestamp. */ + SPECIFIC_TIMESTAMP(KafkaTopicPartitionStateSentinel.TIMESTAMP), + /** * Start from user-supplied specific offsets for each partition. * Since this mode will have specific offsets to start with, we do not need a sentinel value; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java index 68f842ae9c522..1503a99a23345 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java @@ -28,6 +28,9 @@ @Internal public class KafkaTopicPartitionStateSentinel { + /** Magic number that defines the partition should start from specify timestamp. */ + public static final long TIMESTAMP = -915623761777L; + /** Magic number that defines an unset offset. */ public static final long OFFSET_NOT_SET = -915623761776L; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 55a9c4d3b1492..1e26d1f85aacb 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -375,7 +375,7 @@ public void runStartFromEarliestOffsets() throws Exception { kafkaOffsetHandler.setCommittedOffset(topicName, 1, 31); kafkaOffsetHandler.setCommittedOffset(topicName, 2, 43); - readSequence(env, StartupMode.EARLIEST, null, readProps, parallelism, topicName, recordsInEachPartition, 0); + readSequence(env, StartupMode.EARLIEST, null, null, readProps, parallelism, topicName, recordsInEachPartition, 0); kafkaOffsetHandler.close(); deleteTestTopic(topicName); @@ -557,7 +557,7 @@ public void runStartFromGroupOffsets() throws Exception { partitionsToValueCountAndStartOffsets.put(1, new Tuple2<>(50, 0)); // partition 1 should read offset 0-49 partitionsToValueCountAndStartOffsets.put(2, new Tuple2<>(7, 43)); // partition 2 should read offset 43-49 - readSequence(env, StartupMode.GROUP_OFFSETS, null, readProps, topicName, partitionsToValueCountAndStartOffsets); + readSequence(env, StartupMode.GROUP_OFFSETS, null, null, readProps, topicName, partitionsToValueCountAndStartOffsets); kafkaOffsetHandler.close(); deleteTestTopic(topicName); @@ -621,12 +621,40 @@ public void runStartFromSpecificOffsets() throws Exception { partitionsToValueCountAndStartOffsets.put(2, new Tuple2<>(28, 22)); // partition 2 should read offset 22-49 partitionsToValueCountAndStartOffsets.put(3, new Tuple2<>(50, 0)); // partition 3 should read offset 0-49 - readSequence(env, StartupMode.SPECIFIC_OFFSETS, specificStartupOffsets, readProps, topicName, partitionsToValueCountAndStartOffsets); + readSequence(env, StartupMode.SPECIFIC_OFFSETS, specificStartupOffsets, null, readProps, topicName, partitionsToValueCountAndStartOffsets); kafkaOffsetHandler.close(); deleteTestTopic(topicName); } + /** + * This test ensures that the consumer correctly uses user-supplied specific date when explicitly configured to + * start from specific date. + * + *

    When configured to start from first start date, each partition should start from offset 0 and read 100 records. + * And when configured to start from second start date, each partition should start from 50 and read 50 records. + */ + protected void runStartFromSpecificDate() throws Exception { + // 4 partitions with 50 records each + final int parallelism = 4; + final int recordsInEachPartition = 50; + TopicWithStartDate topicWithStartDate = writeAppendSequence("runStartFromSpecificDate", recordsInEachPartition, parallelism, 1); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().disableSysoutLogging(); + env.setParallelism(parallelism); + + Properties readProps = new Properties(); + readProps.putAll(standardProps); + + readSequence(env, StartupMode.SPECIFIC_TIMESTAMP, null, topicWithStartDate.getFirstStartDate(), + readProps, parallelism, topicWithStartDate.getTopicName(), recordsInEachPartition * 2, 0); + readSequence(env, StartupMode.SPECIFIC_TIMESTAMP, null, topicWithStartDate.getSecondStartDate(), + readProps, parallelism, topicWithStartDate.getTopicName(), recordsInEachPartition, recordsInEachPartition); + + deleteTestTopic(topicWithStartDate.getTopicName()); + } + /** * Ensure Kafka is working on both producer and consumer side. * This executes a job that contains two Flink pipelines. @@ -1712,6 +1740,7 @@ public TypeInformation> getProducedType() { protected void readSequence(final StreamExecutionEnvironment env, final StartupMode startupMode, final Map specificStartupOffsets, + final Date specificStartupDate, final Properties cc, final String topicName, final Map> partitionsToValuesCountAndStartOffset) throws Exception { @@ -1731,20 +1760,7 @@ protected void readSequence(final StreamExecutionEnvironment env, // create the consumer cc.putAll(secureProps); FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deser, cc); - switch (startupMode) { - case EARLIEST: - consumer.setStartFromEarliest(); - break; - case LATEST: - consumer.setStartFromLatest(); - break; - case SPECIFIC_OFFSETS: - consumer.setStartFromSpecificOffsets(specificStartupOffsets); - break; - case GROUP_OFFSETS: - consumer.setStartFromGroupOffsets(); - break; - } + setKafkaConsumerOffset(startupMode, consumer, specificStartupOffsets, specificStartupDate); DataStream> source = env .addSource(consumer).setParallelism(sourceParallelism) @@ -1808,12 +1824,13 @@ public void flatMap(Tuple2 value, Collector out) thro } /** - * Variant of {@link KafkaConsumerTestBase#readSequence(StreamExecutionEnvironment, StartupMode, Map, Properties, String, Map)} to + * Variant of {@link KafkaConsumerTestBase#readSequence(StreamExecutionEnvironment, StartupMode, Map, Date, Properties, String, Map)} to * expect reading from the same start offset and the same value count for all partitions of a single Kafka topic. */ protected void readSequence(final StreamExecutionEnvironment env, final StartupMode startupMode, final Map specificStartupOffsets, + final Date specificStartupDate, final Properties cc, final int sourceParallelism, final String topicName, @@ -1823,7 +1840,29 @@ protected void readSequence(final StreamExecutionEnvironment env, for (int i = 0; i < sourceParallelism; i++) { partitionsToValuesCountAndStartOffset.put(i, new Tuple2<>(valuesCount, startFrom)); } - readSequence(env, startupMode, specificStartupOffsets, cc, topicName, partitionsToValuesCountAndStartOffset); + readSequence(env, startupMode, specificStartupOffsets, specificStartupDate, cc, topicName, partitionsToValuesCountAndStartOffset); + } + + protected void setKafkaConsumerOffset(final StartupMode startupMode, + final FlinkKafkaConsumerBase> consumer, + final Map specificStartupOffsets, + final Date specificStartupDate) { + switch (startupMode) { + case EARLIEST: + consumer.setStartFromEarliest(); + break; + case LATEST: + consumer.setStartFromLatest(); + break; + case SPECIFIC_OFFSETS: + consumer.setStartFromSpecificOffsets(specificStartupOffsets); + break; + case GROUP_OFFSETS: + consumer.setStartFromGroupOffsets(); + break; + case SPECIFIC_TIMESTAMP: + throw new RuntimeException("Only support to start from specific start up date for version 0.10 and later"); + } } protected String writeSequence( @@ -1992,6 +2031,246 @@ public void run() { throw new Exception("Could not write a valid sequence to Kafka after " + maxNumAttempts + " attempts"); } + protected TopicWithStartDate writeAppendSequence( + String baseTopicName, + final int numElements, + final int parallelism, + final int replicationFactor) throws Exception { + LOG.info("\n===================================\n" + + "== Writing sequence of " + numElements + " into " + baseTopicName + " with p=" + parallelism + "\n" + + "==================================="); + + final TypeInformation> resultType = + TypeInformation.of(new TypeHint>() {}); + + final KeyedSerializationSchema> serSchema = + new KeyedSerializationSchemaWrapper<>( + new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig())); + + final KeyedDeserializationSchema> deserSchema = + new KeyedDeserializationSchemaWrapper<>( + new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig())); + + final int maxNumAttempts = 10; + + final List> kafkaTupleList = new ArrayList<>(); + for (int attempt = 1; attempt <= maxNumAttempts; attempt++) { + + final String topicName = baseTopicName + '-' + attempt; + + LOG.info("Writing attempt #1"); + + // -------- Write the Sequence -------- + + createTestTopic(topicName, parallelism, replicationFactor); + + Date firstStartDate = new Date(); + StreamExecutionEnvironment writeEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + writeEnv.getConfig().disableSysoutLogging(); + + DataStream> stream = writeEnv.addSource(new RichParallelSourceFunction>() { + + private boolean running = true; + + @Override + public void run(SourceContext> ctx) throws Exception { + int cnt = 0; + int partition = getRuntimeContext().getIndexOfThisSubtask(); + + while (running && cnt < numElements) { + ctx.collect(new Tuple2<>(partition, cnt)); + cnt++; + } + } + + @Override + public void cancel() { + running = false; + } + }).setParallelism(parallelism); + + // the producer must not produce duplicates + Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + producerProperties.setProperty("retries", "0"); + producerProperties.putAll(secureProps); + + kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2FlinkPartitioner(parallelism)) + .setParallelism(parallelism); + + try { + writeEnv.execute("Write sequence"); + } + catch (Exception e) { + LOG.error("Write attempt failed, trying again", e); + deleteTestTopic(topicName); + JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); + continue; + } + + Thread.sleep(10); + Date secondStartDate = new Date(); + writeEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + writeEnv.getConfig().disableSysoutLogging(); + + stream = writeEnv.addSource(new RichParallelSourceFunction>() { + + private boolean running = true; + + @Override + public void run(SourceContext> ctx) throws Exception { + int cnt = numElements; + int partition = getRuntimeContext().getIndexOfThisSubtask(); + + while (running && cnt < numElements + numElements) { + ctx.collect(new Tuple2<>(partition, cnt)); + cnt++; + } + } + + @Override + public void cancel() { + running = false; + } + }).setParallelism(parallelism); + + // the producer must not produce duplicates + producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + producerProperties.setProperty("retries", "0"); + producerProperties.putAll(secureProps); + + kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2FlinkPartitioner(parallelism)) + .setParallelism(parallelism); + + try { + writeEnv.execute("Write sequence"); + } + catch (Exception e) { + LOG.error("Write attempt failed, trying again", e); + deleteTestTopic(topicName); + JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); + continue; + } + + LOG.info("Finished writing sequence"); + + // -------- Validate the Sequence -------- + + // we need to validate the sequence, because kafka's producers are not exactly once + LOG.info("Validating sequence"); + + JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); + + final StreamExecutionEnvironment readEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + readEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + readEnv.getConfig().disableSysoutLogging(); + readEnv.setParallelism(parallelism); + + Properties readProps = (Properties) standardProps.clone(); + readProps.setProperty("group.id", "flink-tests-validator"); + readProps.putAll(secureProps); + FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deserSchema, readProps); + + readEnv + .addSource(consumer) + .map(new RichMapFunction, Tuple2>() { + + private final int totalCount = parallelism * (numElements + 1); + private int count = 0; + + @Override + public Tuple2 map(Tuple2 value) throws Exception { + if (++count == totalCount) { + throw new SuccessException(); + } else { + return value; + } + } + }).setParallelism(1) + .addSink(new DiscardingSink>()).setParallelism(1); + + final AtomicReference errorRef = new AtomicReference<>(); + + Thread runner = new Thread() { + @Override + public void run() { + try { + tryExecute(readEnv, "sequence validation"); + } catch (Throwable t) { + errorRef.set(t); + } + } + }; + runner.start(); + + final long deadline = System.nanoTime() + 10_000_000_000L; + long delay; + while (runner.isAlive() && (delay = deadline - System.nanoTime()) > 0) { + runner.join(delay / 1_000_000L); + } + + boolean success; + + if (runner.isAlive()) { + // did not finish in time, maybe the producer dropped one or more records and + // the validation did not reach the exit point + success = false; + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); + } + else { + Throwable error = errorRef.get(); + if (error != null) { + success = false; + LOG.info("Attempt " + attempt + " failed with exception", error); + } + else { + success = true; + } + } + + JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); + + if (success) { + // everything is good! + return new TopicWithStartDate(topicName, firstStartDate, secondStartDate); + } + else { + deleteTestTopic(topicName); + // fall through the loop + } + } + + throw new Exception("Could not write a valid sequence to Kafka after " + maxNumAttempts + " attempts"); + } + + /** + * Pojo class for consumer with date. + */ + public static class TopicWithStartDate { + private final String topicName; + private final Date firstStartDate; + private final Date secondStartDate; + + public TopicWithStartDate(String topicName, Date firstStartDate, Date secondStartDate) { + this.topicName = topicName; + this.firstStartDate = firstStartDate; + this.secondStartDate = secondStartDate; + } + + public String getTopicName() { + return topicName; + } + + public Date getFirstStartDate() { + return firstStartDate; + } + + public Date getSecondStartDate() { + return secondStartDate; + } + } + // ------------------------------------------------------------------------ // Debugging utilities // ------------------------------------------------------------------------ From 1fcd516a0c55f22d06b4ce3d9bc37fb9d03f0e33 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 11 Jan 2018 14:26:37 +0800 Subject: [PATCH 038/268] [FLINK-6352] [kafka] Further improvements for timestamped-based startup mode 1) Eagerly deterrmin startup offsets when startup mode is TIMESTAMP 2) Remove usage of java Date in API to specify timestamp 3) Make tests more robust and flexible 4) Add documentation for the feature This closes #5282. --- docs/dev/connectors/kafka.md | 7 + .../kafka/FlinkKafkaConsumer010.java | 78 ++-- .../kafka/internal/Kafka010Fetcher.java | 9 +- .../internal/KafkaConsumerCallBridge010.java | 25 -- .../connectors/kafka/Kafka010ITCase.java | 33 +- .../kafka/internal/Kafka010FetcherTest.java | 9 +- .../connectors/kafka/Kafka011ITCase.java | 33 +- .../kafka/FlinkKafkaConsumer08.java | 8 + .../kafka/FlinkKafkaConsumer09.java | 11 +- .../kafka/internal/Kafka09Fetcher.java | 8 +- .../internal/KafkaConsumerCallBridge.java | 3 - .../kafka/internal/KafkaConsumerThread.java | 3 - .../kafka/internal/Kafka09FetcherTest.java | 9 +- .../kafka/FlinkKafkaConsumerBase.java | 128 +++++- .../connectors/kafka/config/StartupMode.java | 8 +- .../KafkaTopicPartitionStateSentinel.java | 3 - .../FlinkKafkaConsumerBaseMigrationTest.java | 7 + .../kafka/FlinkKafkaConsumerBaseTest.java | 10 +- .../kafka/KafkaConsumerTestBase.java | 420 ++++++------------ 19 files changed, 351 insertions(+), 461 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index f28195c190f7c..27fca7a2cd916 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -191,6 +191,7 @@ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEn FlinkKafkaConsumer08 myConsumer = new FlinkKafkaConsumer08<>(...); myConsumer.setStartFromEarliest(); // start from the earliest record possible myConsumer.setStartFromLatest(); // start from the latest record +myConsumer.setStartFromTimestamp(...); // start from specified epoch timestamp (milliseconds) myConsumer.setStartFromGroupOffsets(); // the default behaviour DataStream stream = env.addSource(myConsumer); @@ -204,6 +205,7 @@ val env = StreamExecutionEnvironment.getExecutionEnvironment() val myConsumer = new FlinkKafkaConsumer08[String](...) myConsumer.setStartFromEarliest() // start from the earliest record possible myConsumer.setStartFromLatest() // start from the latest record +myConsumer.setStartFromTimestamp(...) // start from specified epoch timestamp (milliseconds) myConsumer.setStartFromGroupOffsets() // the default behaviour val stream = env.addSource(myConsumer) @@ -221,6 +223,11 @@ All versions of the Flink Kafka Consumer have the above explicit configuration m * `setStartFromEarliest()` / `setStartFromLatest()`: Start from the earliest / latest record. Under these modes, committed offsets in Kafka will be ignored and not used as starting positions. + * `setStartFromTimestamp(long)`: Start from the specified timestamp. For each partition, the record + whose timestamp is larger than or equal to the specified timestamp will be used as the start position. + If a partition's latest record is earlier than the timestamp, the partition will simply be read + from the latest record. Under this mode, committed offsets in Kafka will be ignored and not used as + starting positions. You can also specify the exact offsets the consumer should start from for each partition: diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java index d5bd1653dc3f5..3508d6de37efc 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer010.java @@ -24,7 +24,6 @@ import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.config.StartupMode; import org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher; import org.apache.flink.streaming.connectors.kafka.internal.Kafka010PartitionDiscoverer; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; @@ -36,16 +35,18 @@ import org.apache.flink.util.SerializedValue; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.common.TopicPartition; +import java.util.Collection; import java.util.Collections; -import java.util.Date; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; -import static org.apache.flink.util.Preconditions.checkArgument; - /** * The Flink Kafka Consumer is a streaming data source that pulls a parallel data stream from * Apache Kafka 0.10.x. The consumer can run in multiple parallel instances, each of which will pull @@ -68,8 +69,6 @@ public class FlinkKafkaConsumer010 extends FlinkKafkaConsumer09 { private static final long serialVersionUID = 2324564345203409112L; - private Date specificStartupDate = null; - // ------------------------------------------------------------------------ /** @@ -178,30 +177,6 @@ public FlinkKafkaConsumer010(Pattern subscriptionPattern, KeyedDeserializationSc super(subscriptionPattern, deserializer, props); } - - /** - * Specifies the consumer to start reading partitions from specific date. The specified date must before current timestamp. - * This lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. - * - *

    The consumer will look up the earliest offset whose timestamp is greater than or equal to the specific date from Kafka. - * If there's no such offset, the consumer will use the latest offset to read data from kafka. - * - *

    This method does not effect where partitions are read from when the consumer is restored - * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or - * savepoint, only the offsets in the restored state will be used. - * - * @return The consumer object, to allow function chaining. - */ - public FlinkKafkaConsumer010 setStartFromSpecificDate(Date date) { - Date now = new Date(); - checkArgument(null != date && date.getTime() <= now.getTime(), - "Startup time[" + date + "] must be before current time[" + now + "]."); - this.startupMode = StartupMode.SPECIFIC_TIMESTAMP; - this.specificStartupDate = date; - this.specificStartupOffsets = null; - return this; - } - @Override protected AbstractFetcher createFetcher( SourceContext sourceContext, @@ -233,8 +208,7 @@ public FlinkKafkaConsumer010 setStartFromSpecificDate(Date date) { pollTimeout, runtimeContext.getMetricGroup(), consumerMetricGroup, - useMetrics, - specificStartupDate); + useMetrics); } @Override @@ -245,4 +219,44 @@ protected AbstractPartitionDiscoverer createPartitionDiscoverer( return new Kafka010PartitionDiscoverer(topicsDescriptor, indexOfThisSubtask, numParallelSubtasks, properties); } + + // ------------------------------------------------------------------------ + // Timestamp-based startup + // ------------------------------------------------------------------------ + + @Override + public FlinkKafkaConsumerBase setStartFromTimestamp(long startupOffsetsTimestamp) { + // the purpose of this override is just to publicly expose the method for Kafka 0.10+; + // the base class doesn't publicly expose it since not all Kafka versions support the functionality + return super.setStartFromTimestamp(startupOffsetsTimestamp); + } + + @Override + protected Map fetchOffsetsWithTimestamp( + Collection partitions, + long timestamp) { + + Map partitionOffsetsRequest = new HashMap<>(partitions.size()); + for (KafkaTopicPartition partition : partitions) { + partitionOffsetsRequest.put( + new TopicPartition(partition.getTopic(), partition.getPartition()), + timestamp); + } + + // use a short-lived consumer to fetch the offsets; + // this is ok because this is a one-time operation that happens only on startup + KafkaConsumer consumer = new KafkaConsumer(properties); + + Map result = new HashMap<>(partitions.size()); + for (Map.Entry partitionToOffset : + consumer.offsetsForTimes(partitionOffsetsRequest).entrySet()) { + + result.put( + new KafkaTopicPartition(partitionToOffset.getKey().topic(), partitionToOffset.getKey().partition()), + (partitionToOffset.getValue() == null) ? null : partitionToOffset.getValue().offset()); + } + + consumer.close(); + return result; + } } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java index 9ed15fc272a59..9b9b217b205f0 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010Fetcher.java @@ -32,7 +32,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import java.util.Date; import java.util.Map; import java.util.Properties; @@ -61,8 +60,7 @@ public Kafka010Fetcher( long pollTimeout, MetricGroup subtaskMetricGroup, MetricGroup consumerMetricGroup, - boolean useMetrics, - Date startupDate) throws Exception { + boolean useMetrics) throws Exception { super( sourceContext, assignedPartitionsWithInitialOffsets, @@ -77,8 +75,7 @@ public Kafka010Fetcher( pollTimeout, subtaskMetricGroup, consumerMetricGroup, - useMetrics, - startupDate); + useMetrics); } @Override @@ -98,7 +95,7 @@ protected void emitRecord( */ @Override protected KafkaConsumerCallBridge010 createCallBridge() { - return new KafkaConsumerCallBridge010(startupDate); + return new KafkaConsumerCallBridge010(); } @Override diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java index 2f430c5a1a7b1..5815bfade38de 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge010.java @@ -21,14 +21,10 @@ import org.apache.flink.annotation.Internal; import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; import java.util.Collections; -import java.util.Date; -import java.util.HashMap; import java.util.List; -import java.util.Map; /** * The ConsumerCallBridge simply calls the {@link KafkaConsumer#assign(java.util.Collection)} method. @@ -41,13 +37,6 @@ @Internal public class KafkaConsumerCallBridge010 extends KafkaConsumerCallBridge { - private Date startupDate; - - public KafkaConsumerCallBridge010(Date startupDate) { - super(); - this.startupDate = startupDate; - } - @Override public void assignPartitions(KafkaConsumer consumer, List topicPartitions) throws Exception { consumer.assign(topicPartitions); @@ -62,18 +51,4 @@ public void seekPartitionToBeginning(KafkaConsumer consumer, TopicPartitio public void seekPartitionToEnd(KafkaConsumer consumer, TopicPartition partition) { consumer.seekToEnd(Collections.singletonList(partition)); } - - @Override - public void seekPartitionToDate(KafkaConsumer consumer, TopicPartition partition) { - Map partitionTimestampMap = new HashMap<>(1); - partitionTimestampMap.put(partition, startupDate.getTime()); - - Map topicPartitionOffsetMap = consumer.offsetsForTimes(partitionTimestampMap); - OffsetAndTimestamp offsetAndTimestamp = null == topicPartitionOffsetMap ? null : topicPartitionOffsetMap.get(partition); - if (null == offsetAndTimestamp) { - seekPartitionToEnd(consumer, partition); - } else { - consumer.seek(partition, offsetAndTimestamp.offset()); - } - } } diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java index f821bf66e193b..06f627d7f425a 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka010ITCase.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.core.memory.DataInputView; @@ -35,8 +34,6 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.connectors.kafka.config.StartupMode; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; @@ -48,8 +45,6 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import java.util.Date; -import java.util.Map; /** * IT cases for Kafka 0.10 . @@ -161,8 +156,8 @@ public void testStartFromSpecificOffsets() throws Exception { } @Test(timeout = 60000) - public void testStartFromSpecificDate() throws Exception { - runStartFromSpecificDate(); + public void testStartFromTimestamp() throws Exception { + runStartFromTimestamp(); } // --- offset committing --- @@ -264,30 +259,6 @@ public long extractTimestamp(Long element, long previousElementTimestamp) { deleteTestTopic(topic); } - @Override - protected void setKafkaConsumerOffset(final StartupMode startupMode, - final FlinkKafkaConsumerBase> consumer, - final Map specificStartupOffsets, - final Date specificStartupDate) { - switch (startupMode) { - case EARLIEST: - consumer.setStartFromEarliest(); - break; - case LATEST: - consumer.setStartFromLatest(); - break; - case SPECIFIC_OFFSETS: - consumer.setStartFromSpecificOffsets(specificStartupOffsets); - break; - case GROUP_OFFSETS: - consumer.setStartFromGroupOffsets(); - break; - case SPECIFIC_TIMESTAMP: - ((FlinkKafkaConsumer010>) consumer).setStartFromSpecificDate(specificStartupDate); - break; - } - } - private static class TimestampValidatingOperator extends StreamSink { private static final long serialVersionUID = 1353168781235526806L; diff --git a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java index accf487dc439e..f57fbea67e147 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka010FetcherTest.java @@ -131,8 +131,7 @@ public Void answer(InvocationOnMock invocation) { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false, - null); + false); // ----- run the fetcher ----- @@ -269,8 +268,7 @@ public Void answer(InvocationOnMock invocation) { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false, - null); + false); // ----- run the fetcher ----- @@ -385,8 +383,7 @@ public void testCancellationWhenEmitBlocks() throws Exception { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false, - null); + false); // ----- run the fetcher ----- diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java index 1bbbdb4ab5a6a..fd6eb617a0df3 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka011ITCase.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.TypeInfoParser; import org.apache.flink.core.memory.DataInputView; @@ -35,8 +34,6 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.connectors.kafka.config.StartupMode; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; @@ -49,8 +46,6 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import java.util.Date; -import java.util.Map; import java.util.Optional; /** @@ -169,8 +164,8 @@ public void testStartFromSpecificOffsets() throws Exception { } @Test(timeout = 60000) - public void testStartFromSpecificDate() throws Exception { - runStartFromSpecificDate(); + public void testStartFromTimestamp() throws Exception { + runStartFromTimestamp(); } // --- offset committing --- @@ -274,30 +269,6 @@ public long extractTimestamp(Long element, long previousElementTimestamp) { deleteTestTopic(topic); } - @Override - protected void setKafkaConsumerOffset(final StartupMode startupMode, - final FlinkKafkaConsumerBase> consumer, - final Map specificStartupOffsets, - final Date specificStartupDate) { - switch (startupMode) { - case EARLIEST: - consumer.setStartFromEarliest(); - break; - case LATEST: - consumer.setStartFromLatest(); - break; - case SPECIFIC_OFFSETS: - consumer.setStartFromSpecificOffsets(specificStartupOffsets); - break; - case GROUP_OFFSETS: - consumer.setStartFromGroupOffsets(); - break; - case SPECIFIC_TIMESTAMP: - ((FlinkKafkaConsumer011>) consumer).setStartFromSpecificDate(specificStartupDate); - break; - } - } - private static class TimestampValidatingOperator extends StreamSink { private static final long serialVersionUID = 1353168781235526806L; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java index 312c8c46403ab..86f69cd3f6e2b 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java @@ -37,6 +37,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -274,6 +275,13 @@ protected boolean getIsAutoCommitEnabled() { PropertiesUtil.getLong(kafkaProperties, "auto.commit.interval.ms", 60000) > 0; } + @Override + protected Map fetchOffsetsWithTimestamp(Collection partitions, long timestamp) { + // this should not be reached, since we do not expose the timestamp-based startup feature in version 0.8. + throw new UnsupportedOperationException( + "Fetching partition offsets using timestamps is only supported in Kafka versions 0.10 and above."); + } + // ------------------------------------------------------------------------ // Kafka / ZooKeeper configuration utilities // ------------------------------------------------------------------------ diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java index 4c3fa32cd1111..497003293415f 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java @@ -40,6 +40,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -260,8 +261,7 @@ private FlinkKafkaConsumer09( pollTimeout, runtimeContext.getMetricGroup(), consumerMetricGroup, - useMetrics, - null); + useMetrics); } @Override @@ -279,6 +279,13 @@ protected boolean getIsAutoCommitEnabled() { PropertiesUtil.getLong(properties, ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 5000) > 0; } + @Override + protected Map fetchOffsetsWithTimestamp(Collection partitions, long timestamp) { + // this should not be reached, since we do not expose the timestamp-based startup feature in version 0.9. + throw new UnsupportedOperationException( + "Fetching partition offsets using timestamps is only supported in Kafka versions 0.10 and above."); + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java index 185760bd680ad..dcc67d5b4fb14 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java @@ -40,7 +40,6 @@ import javax.annotation.Nonnull; -import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -72,9 +71,6 @@ public class Kafka09Fetcher extends AbstractFetcher { /** Flag to mark the main work loop as alive. */ private volatile boolean running = true; - /** The specific date used to set the offset of kafka. */ - protected Date startupDate; - // ------------------------------------------------------------------------ public Kafka09Fetcher( @@ -91,8 +87,7 @@ public Kafka09Fetcher( long pollTimeout, MetricGroup subtaskMetricGroup, MetricGroup consumerMetricGroup, - boolean useMetrics, - Date startupDate) throws Exception { + boolean useMetrics) throws Exception { super( sourceContext, assignedPartitionsWithInitialOffsets, @@ -106,7 +101,6 @@ public Kafka09Fetcher( this.deserializer = deserializer; this.handover = new Handover(); - this.startupDate = startupDate; this.consumerThread = new KafkaConsumerThread( LOG, diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java index ac8ef7a0c93f5..b789633333df2 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerCallBridge.java @@ -50,7 +50,4 @@ public void seekPartitionToEnd(KafkaConsumer consumer, TopicPartition part consumer.seekToEnd(partition); } - public void seekPartitionToDate(KafkaConsumer consumer, TopicPartition partition) { - throw new RuntimeException("Seek offset from a specific date is only supported for version 0.10 and later."); - } } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java index 6f8c699f4fd8c..38e8a41d474cf 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/KafkaConsumerThread.java @@ -423,9 +423,6 @@ void reassignPartitions(List> newPartit } else if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.LATEST_OFFSET) { consumerCallBridge.seekPartitionToEnd(consumerTmp, newPartitionState.getKafkaPartitionHandle()); newPartitionState.setOffset(consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1); - } else if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.TIMESTAMP) { - consumerCallBridge.seekPartitionToDate(consumerTmp, newPartitionState.getKafkaPartitionHandle()); - newPartitionState.setOffset(consumerTmp.position(newPartitionState.getKafkaPartitionHandle()) - 1); } else if (newPartitionState.getOffset() == KafkaTopicPartitionStateSentinel.GROUP_OFFSET) { // the KafkaConsumer by default will automatically seek the consumer position // to the committed group offset, so we do not need to do it. diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java index 749c2dbdc4132..27b67f1098413 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09FetcherTest.java @@ -131,8 +131,7 @@ public Void answer(InvocationOnMock invocation) { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false, - null); + false); // ----- run the fetcher ----- @@ -268,8 +267,7 @@ public Void answer(InvocationOnMock invocation) { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false, - null); + false); // ----- run the fetcher ----- @@ -384,8 +382,7 @@ public void testCancellationWhenEmitBlocks() throws Exception { 0L, new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - false, - null); + false); // ----- run the fetcher ----- diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index 94230172deec1..e19772a1f801c 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -57,6 +57,7 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -147,10 +148,13 @@ public abstract class FlinkKafkaConsumerBase extends RichParallelSourceFuncti private final long discoveryIntervalMillis; /** The startup mode for the consumer (default is {@link StartupMode#GROUP_OFFSETS}). */ - protected StartupMode startupMode = StartupMode.GROUP_OFFSETS; + private StartupMode startupMode = StartupMode.GROUP_OFFSETS; /** Specific startup offsets; only relevant when startup mode is {@link StartupMode#SPECIFIC_OFFSETS}. */ - protected Map specificStartupOffsets; + private Map specificStartupOffsets; + + /** Timestamp to determine startup offsets; only relevant when startup mode is {@link StartupMode#TIMESTAMP}. */ + private Long startupOffsetsTimestamp; // ------------------------------------------------------------------------ // runtime state (used individually by each parallel subtask) @@ -335,7 +339,7 @@ public FlinkKafkaConsumerBase setCommitOffsetsOnCheckpoints(boolean commitOnC * Specifies the consumer to start reading from the earliest offset for all partitions. * This lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. * - *

    This method does not effect where partitions are read from when the consumer is restored + *

    This method does not affect where partitions are read from when the consumer is restored * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or * savepoint, only the offsets in the restored state will be used. * @@ -343,6 +347,7 @@ public FlinkKafkaConsumerBase setCommitOffsetsOnCheckpoints(boolean commitOnC */ public FlinkKafkaConsumerBase setStartFromEarliest() { this.startupMode = StartupMode.EARLIEST; + this.startupOffsetsTimestamp = null; this.specificStartupOffsets = null; return this; } @@ -351,7 +356,7 @@ public FlinkKafkaConsumerBase setStartFromEarliest() { * Specifies the consumer to start reading from the latest offset for all partitions. * This lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. * - *

    This method does not effect where partitions are read from when the consumer is restored + *

    This method does not affect where partitions are read from when the consumer is restored * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or * savepoint, only the offsets in the restored state will be used. * @@ -359,6 +364,41 @@ public FlinkKafkaConsumerBase setStartFromEarliest() { */ public FlinkKafkaConsumerBase setStartFromLatest() { this.startupMode = StartupMode.LATEST; + this.startupOffsetsTimestamp = null; + this.specificStartupOffsets = null; + return this; + } + + /** + * Specifies the consumer to start reading partitions from a specified timestamp. + * The specified timestamp must be before the current timestamp. + * This lets the consumer ignore any committed group offsets in Zookeeper / Kafka brokers. + * + *

    The consumer will look up the earliest offset whose timestamp is greater than or equal + * to the specific timestamp from Kafka. If there's no such offset, the consumer will use the + * latest offset to read data from kafka. + * + *

    This method does not affect where partitions are read from when the consumer is restored + * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or + * savepoint, only the offsets in the restored state will be used. + * + * @param startupOffsetsTimestamp timestamp for the startup offsets, as milliseconds from epoch. + * + * @return The consumer object, to allow function chaining. + */ + // NOTE - + // This method is implemented in the base class because this is where the startup logging and verifications live. + // However, it is not publicly exposed since only newer Kafka versions support the functionality. + // Version-specific subclasses which can expose the functionality should override and allow public access. + protected FlinkKafkaConsumerBase setStartFromTimestamp(long startupOffsetsTimestamp) { + checkArgument(startupOffsetsTimestamp >= 0, "The provided value for the startup offsets timestamp is invalid."); + + long currentTimestamp = System.currentTimeMillis(); + checkArgument(startupOffsetsTimestamp <= currentTimestamp, + "Startup time[%s] must be before current time[%s].", startupOffsetsTimestamp, currentTimestamp); + + this.startupMode = StartupMode.TIMESTAMP; + this.startupOffsetsTimestamp = startupOffsetsTimestamp; this.specificStartupOffsets = null; return this; } @@ -369,7 +409,7 @@ public FlinkKafkaConsumerBase setStartFromLatest() { * properties. If no offset can be found for a partition, the behaviour in "auto.offset.reset" * set in the configuration properties will be used for the partition. * - *

    This method does not effect where partitions are read from when the consumer is restored + *

    This method does not affect where partitions are read from when the consumer is restored * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or * savepoint, only the offsets in the restored state will be used. * @@ -377,6 +417,7 @@ public FlinkKafkaConsumerBase setStartFromLatest() { */ public FlinkKafkaConsumerBase setStartFromGroupOffsets() { this.startupMode = StartupMode.GROUP_OFFSETS; + this.startupOffsetsTimestamp = null; this.specificStartupOffsets = null; return this; } @@ -395,7 +436,7 @@ public FlinkKafkaConsumerBase setStartFromGroupOffsets() { * offsets but still no group offset could be found for it, then the "auto.offset.reset" behaviour set in the * configuration properties will be used for the partition * - *

    This method does not effect where partitions are read from when the consumer is restored + *

    This method does not affect where partitions are read from when the consumer is restored * from a checkpoint or savepoint. When the consumer is restored from a checkpoint or * savepoint, only the offsets in the restored state will be used. * @@ -403,6 +444,7 @@ public FlinkKafkaConsumerBase setStartFromGroupOffsets() { */ public FlinkKafkaConsumerBase setStartFromSpecificOffsets(Map specificStartupOffsets) { this.startupMode = StartupMode.SPECIFIC_OFFSETS; + this.startupOffsetsTimestamp = null; this.specificStartupOffsets = checkNotNull(specificStartupOffsets); return this; } @@ -457,28 +499,57 @@ public void open(Configuration configuration) throws Exception { getRuntimeContext().getIndexOfThisSubtask(), subscribedPartitionsToStartOffsets.size(), subscribedPartitionsToStartOffsets); } else { // use the partition discoverer to fetch the initial seed partitions, - // and set their initial offsets depending on the startup mode - for (KafkaTopicPartition seedPartition : allPartitions) { - if (startupMode != StartupMode.SPECIFIC_OFFSETS) { - subscribedPartitionsToStartOffsets.put(seedPartition, startupMode.getStateSentinel()); - } else { + // and set their initial offsets depending on the startup mode. + // for SPECIFIC_OFFSETS and TIMESTAMP modes, we set the specific offsets now; + // for other modes (EARLIEST, LATEST, and GROUP_OFFSETS), the offset is lazily determined + // when the partition is actually read. + switch (startupMode) { + case SPECIFIC_OFFSETS: if (specificStartupOffsets == null) { - throw new IllegalArgumentException( + throw new IllegalStateException( "Startup mode for the consumer set to " + StartupMode.SPECIFIC_OFFSETS + - ", but no specific offsets were specified"); + ", but no specific offsets were specified."); } - Long specificOffset = specificStartupOffsets.get(seedPartition); - if (specificOffset != null) { - // since the specified offsets represent the next record to read, we subtract - // it by one so that the initial state of the consumer will be correct - subscribedPartitionsToStartOffsets.put(seedPartition, specificOffset - 1); - } else { - // default to group offset behaviour if the user-provided specific offsets - // do not contain a value for this partition - subscribedPartitionsToStartOffsets.put(seedPartition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET); + for (KafkaTopicPartition seedPartition : allPartitions) { + Long specificOffset = specificStartupOffsets.get(seedPartition); + if (specificOffset != null) { + // since the specified offsets represent the next record to read, we subtract + // it by one so that the initial state of the consumer will be correct + subscribedPartitionsToStartOffsets.put(seedPartition, specificOffset - 1); + } else { + // default to group offset behaviour if the user-provided specific offsets + // do not contain a value for this partition + subscribedPartitionsToStartOffsets.put(seedPartition, KafkaTopicPartitionStateSentinel.GROUP_OFFSET); + } + } + + break; + case TIMESTAMP: + if (startupOffsetsTimestamp == null) { + throw new IllegalStateException( + "Startup mode for the consumer set to " + StartupMode.TIMESTAMP + + ", but no startup timestamp was specified."); + } + + for (Map.Entry partitionToOffset + : fetchOffsetsWithTimestamp(allPartitions, startupOffsetsTimestamp).entrySet()) { + subscribedPartitionsToStartOffsets.put( + partitionToOffset.getKey(), + (partitionToOffset.getValue() == null) + // if an offset cannot be retrieved for a partition with the given timestamp, + // we default to using the latest offset for the partition + ? KafkaTopicPartitionStateSentinel.LATEST_OFFSET + // since the specified offsets represent the next record to read, we subtract + // it by one so that the initial state of the consumer will be correct + : partitionToOffset.getValue() - 1); + } + + break; + default: + for (KafkaTopicPartition seedPartition : allPartitions) { + subscribedPartitionsToStartOffsets.put(seedPartition, startupMode.getStateSentinel()); } - } } if (!subscribedPartitionsToStartOffsets.isEmpty()) { @@ -495,6 +566,13 @@ public void open(Configuration configuration) throws Exception { subscribedPartitionsToStartOffsets.size(), subscribedPartitionsToStartOffsets.keySet()); break; + case TIMESTAMP: + LOG.info("Consumer subtask {} will start reading the following {} partitions from timestamp {}: {}", + getRuntimeContext().getIndexOfThisSubtask(), + subscribedPartitionsToStartOffsets.size(), + startupOffsetsTimestamp, + subscribedPartitionsToStartOffsets.keySet()); + break; case SPECIFIC_OFFSETS: LOG.info("Consumer subtask {} will start reading the following {} partitions from the specified startup offsets {}: {}", getRuntimeContext().getIndexOfThisSubtask(), @@ -873,6 +951,10 @@ protected abstract AbstractPartitionDiscoverer createPartitionDiscoverer( protected abstract boolean getIsAutoCommitEnabled(); + protected abstract Map fetchOffsetsWithTimestamp( + Collection partitions, + long timestamp); + // ------------------------------------------------------------------------ // ResultTypeQueryable methods // ------------------------------------------------------------------------ diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java index ec0a7e8dd0e30..d417fdcfdf455 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/config/StartupMode.java @@ -35,8 +35,12 @@ public enum StartupMode { /** Start from the latest offset. */ LATEST(KafkaTopicPartitionStateSentinel.LATEST_OFFSET), - /** Start from specific timestamp. */ - SPECIFIC_TIMESTAMP(KafkaTopicPartitionStateSentinel.TIMESTAMP), + /** + * Start from user-supplied timestamp for each partition. + * Since this mode will have specific offsets to start with, we do not need a sentinel value; + * using Long.MIN_VALUE as a placeholder. + */ + TIMESTAMP(Long.MIN_VALUE), /** * Start from user-supplied specific offsets for each partition. diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java index 1503a99a23345..68f842ae9c522 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionStateSentinel.java @@ -28,9 +28,6 @@ @Internal public class KafkaTopicPartitionStateSentinel { - /** Magic number that defines the partition should start from specify timestamp. */ - public static final long TIMESTAMP = -915623761777L; - /** Magic number that defines an unset offset. */ public static final long OFFSET_NOT_SET = -915623761776L; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java index d9a9e41cdbc3c..768ac16547c42 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java @@ -419,6 +419,13 @@ protected AbstractPartitionDiscoverer createPartitionDiscoverer( protected boolean getIsAutoCommitEnabled() { return false; } + + @Override + protected Map fetchOffsetsWithTimestamp( + Collection partitions, + long timestamp) { + throw new UnsupportedOperationException(); + } } private abstract static class DummySourceContext diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index 263ed8eca7c66..b226ff1360a60 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -67,6 +67,7 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -674,7 +675,14 @@ protected AbstractPartitionDiscoverer createPartitionDiscoverer( @Override protected boolean getIsAutoCommitEnabled() { - return this.isAutoCommitEnabled; + return isAutoCommitEnabled; + } + + @Override + protected Map fetchOffsetsWithTimestamp( + Collection partitions, + long timestamp) { + throw new UnsupportedOperationException(); } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 1e26d1f85aacb..f07c0bb48a562 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -628,17 +628,32 @@ public void runStartFromSpecificOffsets() throws Exception { } /** - * This test ensures that the consumer correctly uses user-supplied specific date when explicitly configured to - * start from specific date. + * This test ensures that the consumer correctly uses user-supplied timestamp when explicitly configured to + * start from timestamp. * - *

    When configured to start from first start date, each partition should start from offset 0 and read 100 records. - * And when configured to start from second start date, each partition should start from 50 and read 50 records. + *

    The validated Kafka data is written in 2 steps: first, an initial 50 records is written to each partition. + * After that, another 30 records is appended to each partition. Before each step, a timestamp is recorded. + * For the validation, when the read job is configured to start from the first timestamp, each partition should start + * from offset 0 and read a total of 80 records. When configured to start from the second timestamp, + * each partition should start from offset 50 and read on the remaining 30 appended records. */ - protected void runStartFromSpecificDate() throws Exception { + public void runStartFromTimestamp() throws Exception { // 4 partitions with 50 records each final int parallelism = 4; - final int recordsInEachPartition = 50; - TopicWithStartDate topicWithStartDate = writeAppendSequence("runStartFromSpecificDate", recordsInEachPartition, parallelism, 1); + final int initialRecordsInEachPartition = 50; + final int appendRecordsInEachPartition = 30; + + // attempt to create an appended test sequence, where the timestamp of writing the appended sequence + // is assured to be larger than the timestamp of the original sequence. + long firstTimestamp = System.currentTimeMillis(); + String topic = writeSequence("runStartFromTimestamp", initialRecordsInEachPartition, parallelism, 1); + + long secondTimestamp = 0; + while (secondTimestamp <= firstTimestamp) { + Thread.sleep(1000); + secondTimestamp = System.currentTimeMillis(); + } + writeAppendSequence(topic, initialRecordsInEachPartition, appendRecordsInEachPartition, parallelism); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.getConfig().disableSysoutLogging(); @@ -647,12 +662,12 @@ protected void runStartFromSpecificDate() throws Exception { Properties readProps = new Properties(); readProps.putAll(standardProps); - readSequence(env, StartupMode.SPECIFIC_TIMESTAMP, null, topicWithStartDate.getFirstStartDate(), - readProps, parallelism, topicWithStartDate.getTopicName(), recordsInEachPartition * 2, 0); - readSequence(env, StartupMode.SPECIFIC_TIMESTAMP, null, topicWithStartDate.getSecondStartDate(), - readProps, parallelism, topicWithStartDate.getTopicName(), recordsInEachPartition, recordsInEachPartition); + readSequence(env, StartupMode.TIMESTAMP, null, firstTimestamp, + readProps, parallelism, topic, initialRecordsInEachPartition + appendRecordsInEachPartition, 0); + readSequence(env, StartupMode.TIMESTAMP, null, secondTimestamp, + readProps, parallelism, topic, appendRecordsInEachPartition, initialRecordsInEachPartition); - deleteTestTopic(topicWithStartDate.getTopicName()); + deleteTestTopic(topic); } /** @@ -1740,7 +1755,7 @@ public TypeInformation> getProducedType() { protected void readSequence(final StreamExecutionEnvironment env, final StartupMode startupMode, final Map specificStartupOffsets, - final Date specificStartupDate, + final Long startupTimestamp, final Properties cc, final String topicName, final Map> partitionsToValuesCountAndStartOffset) throws Exception { @@ -1760,7 +1775,7 @@ protected void readSequence(final StreamExecutionEnvironment env, // create the consumer cc.putAll(secureProps); FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deser, cc); - setKafkaConsumerOffset(startupMode, consumer, specificStartupOffsets, specificStartupDate); + setKafkaConsumerOffset(startupMode, consumer, specificStartupOffsets, startupTimestamp); DataStream> source = env .addSource(consumer).setParallelism(sourceParallelism) @@ -1824,13 +1839,13 @@ public void flatMap(Tuple2 value, Collector out) thro } /** - * Variant of {@link KafkaConsumerTestBase#readSequence(StreamExecutionEnvironment, StartupMode, Map, Date, Properties, String, Map)} to + * Variant of {@link KafkaConsumerTestBase#readSequence(StreamExecutionEnvironment, StartupMode, Map, Long, Properties, String, Map)} to * expect reading from the same start offset and the same value count for all partitions of a single Kafka topic. */ protected void readSequence(final StreamExecutionEnvironment env, final StartupMode startupMode, final Map specificStartupOffsets, - final Date specificStartupDate, + final Long startupTimestamp, final Properties cc, final int sourceParallelism, final String topicName, @@ -1840,13 +1855,13 @@ protected void readSequence(final StreamExecutionEnvironment env, for (int i = 0; i < sourceParallelism; i++) { partitionsToValuesCountAndStartOffset.put(i, new Tuple2<>(valuesCount, startFrom)); } - readSequence(env, startupMode, specificStartupOffsets, specificStartupDate, cc, topicName, partitionsToValuesCountAndStartOffset); + readSequence(env, startupMode, specificStartupOffsets, startupTimestamp, cc, topicName, partitionsToValuesCountAndStartOffset); } protected void setKafkaConsumerOffset(final StartupMode startupMode, final FlinkKafkaConsumerBase> consumer, final Map specificStartupOffsets, - final Date specificStartupDate) { + final Long startupTimestamp) { switch (startupMode) { case EARLIEST: consumer.setStartFromEarliest(); @@ -1860,8 +1875,9 @@ protected void setKafkaConsumerOffset(final StartupMode startupMode, case GROUP_OFFSETS: consumer.setStartFromGroupOffsets(); break; - case SPECIFIC_TIMESTAMP: - throw new RuntimeException("Only support to start from specific start up date for version 0.10 and later"); + case TIMESTAMP: + consumer.setStartFromTimestamp(startupTimestamp); + break; } } @@ -1891,7 +1907,7 @@ protected String writeSequence( final String topicName = baseTopicName + '-' + attempt; - LOG.info("Writing attempt #1"); + LOG.info("Writing attempt #" + attempt); // -------- Write the Sequence -------- @@ -1949,76 +1965,7 @@ public void cancel() { JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - final StreamExecutionEnvironment readEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - readEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); - readEnv.getConfig().disableSysoutLogging(); - readEnv.setParallelism(parallelism); - - Properties readProps = (Properties) standardProps.clone(); - readProps.setProperty("group.id", "flink-tests-validator"); - readProps.putAll(secureProps); - FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deserSchema, readProps); - - readEnv - .addSource(consumer) - .map(new RichMapFunction, Tuple2>() { - - private final int totalCount = parallelism * numElements; - private int count = 0; - - @Override - public Tuple2 map(Tuple2 value) throws Exception { - if (++count == totalCount) { - throw new SuccessException(); - } else { - return value; - } - } - }).setParallelism(1) - .addSink(new DiscardingSink>()).setParallelism(1); - - final AtomicReference errorRef = new AtomicReference<>(); - - Thread runner = new Thread() { - @Override - public void run() { - try { - tryExecute(readEnv, "sequence validation"); - } catch (Throwable t) { - errorRef.set(t); - } - } - }; - runner.start(); - - final long deadline = System.nanoTime() + 10_000_000_000L; - long delay; - while (runner.isAlive() && (delay = deadline - System.nanoTime()) > 0) { - runner.join(delay / 1_000_000L); - } - - boolean success; - - if (runner.isAlive()) { - // did not finish in time, maybe the producer dropped one or more records and - // the validation did not reach the exit point - success = false; - JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); - } - else { - Throwable error = errorRef.get(); - if (error != null) { - success = false; - LOG.info("Attempt " + attempt + " failed with exception", error); - } - else { - success = true; - } - } - - JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - - if (success) { + if (validateSequence(topicName, parallelism, deserSchema, numElements)) { // everything is good! return topicName; } @@ -2031,13 +1978,14 @@ public void run() { throw new Exception("Could not write a valid sequence to Kafka after " + maxNumAttempts + " attempts"); } - protected TopicWithStartDate writeAppendSequence( - String baseTopicName, - final int numElements, - final int parallelism, - final int replicationFactor) throws Exception { + protected void writeAppendSequence( + String topicName, + final int originalNumElements, + final int numElementsToAppend, + final int parallelism) throws Exception { + LOG.info("\n===================================\n" + - "== Writing sequence of " + numElements + " into " + baseTopicName + " with p=" + parallelism + "\n" + + "== Appending sequence of " + numElementsToAppend + " into " + topicName + "==================================="); final TypeInformation> resultType = @@ -2051,224 +1999,136 @@ protected TopicWithStartDate writeAppendSequence( new KeyedDeserializationSchemaWrapper<>( new TypeInformationSerializationSchema<>(resultType, new ExecutionConfig())); - final int maxNumAttempts = 10; - - final List> kafkaTupleList = new ArrayList<>(); - for (int attempt = 1; attempt <= maxNumAttempts; attempt++) { - - final String topicName = baseTopicName + '-' + attempt; - - LOG.info("Writing attempt #1"); - - // -------- Write the Sequence -------- - - createTestTopic(topicName, parallelism, replicationFactor); - - Date firstStartDate = new Date(); - StreamExecutionEnvironment writeEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); - writeEnv.getConfig().disableSysoutLogging(); + // -------- Write the append sequence -------- - DataStream> stream = writeEnv.addSource(new RichParallelSourceFunction>() { + StreamExecutionEnvironment writeEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + writeEnv.getConfig().disableSysoutLogging(); - private boolean running = true; + DataStream> stream = writeEnv.addSource(new RichParallelSourceFunction>() { - @Override - public void run(SourceContext> ctx) throws Exception { - int cnt = 0; - int partition = getRuntimeContext().getIndexOfThisSubtask(); + private boolean running = true; - while (running && cnt < numElements) { - ctx.collect(new Tuple2<>(partition, cnt)); - cnt++; - } - } + @Override + public void run(SourceContext> ctx) throws Exception { + int cnt = originalNumElements; + int partition = getRuntimeContext().getIndexOfThisSubtask(); - @Override - public void cancel() { - running = false; + while (running && cnt < numElementsToAppend + originalNumElements) { + ctx.collect(new Tuple2<>(partition, cnt)); + cnt++; } - }).setParallelism(parallelism); - - // the producer must not produce duplicates - Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); - producerProperties.setProperty("retries", "0"); - producerProperties.putAll(secureProps); - - kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2FlinkPartitioner(parallelism)) - .setParallelism(parallelism); - - try { - writeEnv.execute("Write sequence"); } - catch (Exception e) { - LOG.error("Write attempt failed, trying again", e); - deleteTestTopic(topicName); - JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - continue; - } - - Thread.sleep(10); - Date secondStartDate = new Date(); - writeEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - writeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); - writeEnv.getConfig().disableSysoutLogging(); - - stream = writeEnv.addSource(new RichParallelSourceFunction>() { - - private boolean running = true; - @Override - public void run(SourceContext> ctx) throws Exception { - int cnt = numElements; - int partition = getRuntimeContext().getIndexOfThisSubtask(); + @Override + public void cancel() { + running = false; + } + }).setParallelism(parallelism); - while (running && cnt < numElements + numElements) { - ctx.collect(new Tuple2<>(partition, cnt)); - cnt++; - } - } + // the producer must not produce duplicates + Properties producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); + producerProperties.setProperty("retries", "0"); + producerProperties.putAll(secureProps); - @Override - public void cancel() { - running = false; - } - }).setParallelism(parallelism); + kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2FlinkPartitioner(parallelism)) + .setParallelism(parallelism); - // the producer must not produce duplicates - producerProperties = FlinkKafkaProducerBase.getPropertiesFromBrokerList(brokerConnectionStrings); - producerProperties.setProperty("retries", "0"); - producerProperties.putAll(secureProps); + try { + writeEnv.execute("Write sequence"); + } + catch (Exception e) { + throw new Exception("Failed to append sequence to Kafka; append job failed.", e); + } - kafkaServer.produceIntoKafka(stream, topicName, serSchema, producerProperties, new Tuple2FlinkPartitioner(parallelism)) - .setParallelism(parallelism); + LOG.info("Finished writing append sequence"); - try { - writeEnv.execute("Write sequence"); - } - catch (Exception e) { - LOG.error("Write attempt failed, trying again", e); - deleteTestTopic(topicName); - JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - continue; - } + // we need to validate the sequence, because kafka's producers are not exactly once + LOG.info("Validating sequence"); + JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - LOG.info("Finished writing sequence"); + if (!validateSequence(topicName, parallelism, deserSchema, originalNumElements + numElementsToAppend)) { + throw new Exception("Could not append a valid sequence to Kafka."); + } + } - // -------- Validate the Sequence -------- + private boolean validateSequence( + final String topic, + final int parallelism, + KeyedDeserializationSchema> deserSchema, + final int totalNumElements) throws Exception { - // we need to validate the sequence, because kafka's producers are not exactly once - LOG.info("Validating sequence"); + final StreamExecutionEnvironment readEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + readEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + readEnv.getConfig().disableSysoutLogging(); + readEnv.setParallelism(parallelism); - JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); + Properties readProps = (Properties) standardProps.clone(); + readProps.setProperty("group.id", "flink-tests-validator"); + readProps.putAll(secureProps); + FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topic, deserSchema, readProps); + consumer.setStartFromEarliest(); - final StreamExecutionEnvironment readEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - readEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); - readEnv.getConfig().disableSysoutLogging(); - readEnv.setParallelism(parallelism); - - Properties readProps = (Properties) standardProps.clone(); - readProps.setProperty("group.id", "flink-tests-validator"); - readProps.putAll(secureProps); - FlinkKafkaConsumerBase> consumer = kafkaServer.getConsumer(topicName, deserSchema, readProps); - - readEnv - .addSource(consumer) - .map(new RichMapFunction, Tuple2>() { - - private final int totalCount = parallelism * (numElements + 1); - private int count = 0; - - @Override - public Tuple2 map(Tuple2 value) throws Exception { - if (++count == totalCount) { - throw new SuccessException(); - } else { - return value; - } - } - }).setParallelism(1) - .addSink(new DiscardingSink>()).setParallelism(1); + readEnv + .addSource(consumer) + .map(new RichMapFunction, Tuple2>() { - final AtomicReference errorRef = new AtomicReference<>(); + private final int totalCount = parallelism * totalNumElements; + private int count = 0; - Thread runner = new Thread() { @Override - public void run() { - try { - tryExecute(readEnv, "sequence validation"); - } catch (Throwable t) { - errorRef.set(t); + public Tuple2 map(Tuple2 value) throws Exception { + if (++count == totalCount) { + throw new SuccessException(); + } else { + return value; } } - }; - runner.start(); - - final long deadline = System.nanoTime() + 10_000_000_000L; - long delay; - while (runner.isAlive() && (delay = deadline - System.nanoTime()) > 0) { - runner.join(delay / 1_000_000L); - } + }).setParallelism(1) + .addSink(new DiscardingSink<>()).setParallelism(1); - boolean success; + final AtomicReference errorRef = new AtomicReference<>(); - if (runner.isAlive()) { - // did not finish in time, maybe the producer dropped one or more records and - // the validation did not reach the exit point - success = false; - JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); - } - else { - Throwable error = errorRef.get(); - if (error != null) { - success = false; - LOG.info("Attempt " + attempt + " failed with exception", error); - } - else { - success = true; + Thread runner = new Thread() { + @Override + public void run() { + try { + tryExecute(readEnv, "sequence validation"); + } catch (Throwable t) { + errorRef.set(t); } } + }; + runner.start(); - JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - - if (success) { - // everything is good! - return new TopicWithStartDate(topicName, firstStartDate, secondStartDate); - } - else { - deleteTestTopic(topicName); - // fall through the loop - } + final long deadline = System.nanoTime() + 10_000_000_000L; + long delay; + while (runner.isAlive() && (delay = deadline - System.nanoTime()) > 0) { + runner.join(delay / 1_000_000L); } - throw new Exception("Could not write a valid sequence to Kafka after " + maxNumAttempts + " attempts"); - } + boolean success; - /** - * Pojo class for consumer with date. - */ - public static class TopicWithStartDate { - private final String topicName; - private final Date firstStartDate; - private final Date secondStartDate; - - public TopicWithStartDate(String topicName, Date firstStartDate, Date secondStartDate) { - this.topicName = topicName; - this.firstStartDate = firstStartDate; - this.secondStartDate = secondStartDate; + if (runner.isAlive()) { + // did not finish in time, maybe the producer dropped one or more records and + // the validation did not reach the exit point + success = false; + JobManagerCommunicationUtils.cancelCurrentJob(flink.getLeaderGateway(timeout)); } - - public String getTopicName() { - return topicName; + else { + Throwable error = errorRef.get(); + if (error != null) { + success = false; + LOG.info("Sequence validation job failed with exception", error); + } + else { + success = true; + } } - public Date getFirstStartDate() { - return firstStartDate; - } + JobManagerCommunicationUtils.waitUntilNoJobIsRunning(flink.getLeaderGateway(timeout)); - public Date getSecondStartDate() { - return secondStartDate; - } + return success; } // ------------------------------------------------------------------------ From 02cb9e39cc19ebbf82f06d87035aead20780eb2b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Feb 2018 14:07:12 +0100 Subject: [PATCH 039/268] [FLINK-8808] [flip6] Allow RestClusterClient to connect to local dispatcher The RestClusterClient resolves a dispatcher address without an explicit host to 'localhost'. That way we allow the RestClusterClient to talk to a Dispatcher which runs in a local ActorSystem. This closes #5599. --- .../program/rest/RestClusterClient.java | 20 ++++-- .../apache/flink/runtime/util/ScalaUtils.java | 46 +++++++++++++ .../flink/runtime/util/ScalaUtilsTest.java | 66 +++++++++++++++++++ 3 files changed, 125 insertions(+), 7 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/util/ScalaUtils.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/util/ScalaUtilsTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 3a377f3ef49b1..98332968004bc 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -77,6 +77,7 @@ import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.runtime.util.ScalaUtils; import org.apache.flink.runtime.webmonitor.retriever.LeaderRetriever; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; @@ -97,6 +98,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; @@ -107,10 +109,6 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -import scala.Option; - -import static org.apache.flink.util.Preconditions.checkArgument; - /** * A {@link ClusterClient} implementation that communicates via HTTP REST requests. */ @@ -636,9 +634,17 @@ private CompletableFuture getDispatcherAddress() { TimeUnit.MILLISECONDS) .thenApplyAsync(leaderAddressSessionId -> { final String address = leaderAddressSessionId.f0; - final Option host = AddressFromURIString.parse(address).host(); - checkArgument(host.isDefined(), "Could not parse host from %s", address); - return host.get(); + final Optional host = ScalaUtils.toJava(AddressFromURIString.parse(address).host()); + + return host.orElseGet(() -> { + // if the dispatcher address does not contain a host part, then assume it's running + // on the same machine as the client + log.info("The dispatcher seems to run without remoting enabled. This indicates that we are " + + "in a test. This can only work if the RestClusterClient runs on the same machine. " + + "Assuming, therefore, 'localhost' as the host."); + + return "localhost"; + }); }, executorService); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ScalaUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ScalaUtils.java new file mode 100644 index 0000000000000..fa4260eded8fa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ScalaUtils.java @@ -0,0 +1,46 @@ +/* + * 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.util; + +import java.util.Optional; + +import scala.Option; + +/** + * Utilities to convert Scala types into Java types. + */ +public class ScalaUtils { + + /** + * Converts a Scala {@link Option} to a {@link Optional}. + * + * @param scalaOption to convert into ta Java {@link Optional} + * @param type of the optional value + * @return Optional of the given option + */ + public static Optional toJava(Option scalaOption) { + if (scalaOption.isEmpty()) { + return Optional.empty(); + } else { + return Optional.ofNullable(scalaOption.get()); + } + } + + private ScalaUtils() {} +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/ScalaUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ScalaUtilsTest.java new file mode 100644 index 0000000000000..e5e9896240d1a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/ScalaUtilsTest.java @@ -0,0 +1,66 @@ +/* + * 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.util; + +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.NoSuchElementException; +import java.util.Optional; + +import scala.Option; + +import static org.apache.flink.runtime.util.ScalaUtils.toJava; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +/** + * Tests for {@link ScalaUtils} convenience methods. + */ +public class ScalaUtilsTest extends TestLogger { + + @Test + public void testOptionToOptional() { + final String value = "foobar"; + final Option option = Option.apply(value); + final Optional optional = toJava(option); + + assertThat(optional.isPresent(), is(true)); + assertThat(optional.get(), is(value)); + + final Option nullOption = Option.apply(null); + final Optional nullOptional = toJava(nullOption); + + assertThat(nullOptional.isPresent(), is(false)); + + try { + nullOptional.get(); + fail("Expected NoSuchElementException"); + } catch (NoSuchElementException ignored) { + // ignored + } + + final Option emptyOption = Option.empty(); + final Optional emptyOptional = toJava(emptyOption); + + assertThat(emptyOptional.isPresent(), is(false)); + } +} From f30ca2101f2151214b138f37f472f886fbdfd9f0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Feb 2018 17:36:39 +0100 Subject: [PATCH 040/268] [FLINK-8811] [flip6] Add initial implementation of the MiniClusterClient The MiniClusterClient directly talks to the MiniCluster avoiding polling latencies of th RestClusterClient. This closes #5600. --- .../client/program/MiniClusterClient.java | 171 ++++++++++++++++++ .../runtime/minicluster/MiniCluster.java | 7 + .../flink/test/util/MiniClusterResource.java | 53 ++++-- 3 files changed, 216 insertions(+), 15 deletions(-) create mode 100644 flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java new file mode 100644 index 0000000000000..5baae5b0a3e63 --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -0,0 +1,171 @@ +/* + * 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.client.program; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.util.LeaderConnectionInfo; +import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.util.FlinkException; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.net.URL; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * Client to interact with a {@link MiniCluster}. + */ +public class MiniClusterClient extends ClusterClient { + + private final MiniCluster miniCluster; + + public MiniClusterClient(@Nonnull Configuration configuration, @Nonnull MiniCluster miniCluster) throws Exception { + super(configuration, miniCluster.getHighAvailabilityServices()); + + this.miniCluster = miniCluster; + } + + @Override + protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { + if (isDetached()) { + try { + miniCluster.runDetached(jobGraph); + } catch (JobExecutionException | InterruptedException e) { + throw new ProgramInvocationException( + String.format("Could not run job %s in detached mode.", jobGraph.getJobID()), + e); + } + + return new JobSubmissionResult(jobGraph.getJobID()); + } else { + try { + return miniCluster.executeJobBlocking(jobGraph); + } catch (JobExecutionException | InterruptedException e) { + throw new ProgramInvocationException( + String.format("Could not run job %s.", jobGraph.getJobID()), + e); + } + } + } + + @Override + public void cancel(JobID jobId) throws Exception { + throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + } + + @Override + public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) throws Exception { + throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + } + + @Override + public void stop(JobID jobId) throws Exception { + throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + } + + @Override + public CompletableFuture triggerSavepoint(JobID jobId, @Nullable String savepointDirectory) throws FlinkException { + throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + } + + @Override + public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) throws FlinkException { + throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + } + + @Override + public CompletableFuture> listJobs() throws Exception { + throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + } + + @Override + public Map getAccumulators(JobID jobID) throws Exception { + throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + } + + @Override + public Map getAccumulators(JobID jobID, ClassLoader loader) throws Exception { + throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + } + + @Override + public MiniClusterClient.MiniClusterId getClusterId() { + return MiniClusterId.INSTANCE; + } + + @Override + public LeaderConnectionInfo getClusterConnectionInfo() throws LeaderRetrievalException { + return LeaderRetrievalUtils.retrieveLeaderConnectionInfo( + highAvailabilityServices.getDispatcherLeaderRetriever(), + timeout); + } + + // ====================================== + // Legacy methods + // ====================================== + + @Override + public void waitForClusterToBeReady() { + // no op + } + + @Override + public String getWebInterfaceURL() { + return miniCluster.getRestAddress().toString(); + } + + @Override + public GetClusterStatusResponse getClusterStatus() { + return null; + } + + @Override + public List getNewMessages() { + return Collections.emptyList(); + } + + @Override + public int getMaxSlots() { + return 0; + } + + @Override + public boolean hasUserJarsInClassPath(List userJarFiles) { + return false; + } + + enum MiniClusterId { + INSTANCE + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 3bd17d1e85824..cbfb266499287 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -178,6 +178,13 @@ public URI getRestAddress() { } } + public HighAvailabilityServices getHighAvailabilityServices() { + synchronized (lock) { + checkState(running, "MiniCluster is not yet running."); + return haServices; + } + } + // ------------------------------------------------------------------------ // life cycle // ------------------------------------------------------------------------ diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java index 1c5da62b2e000..a1ce64737571a 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java @@ -19,15 +19,20 @@ package org.apache.flink.test.util; import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.MiniClusterClient; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.minicluster.JobExecutorService; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; @@ -35,8 +40,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; - import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -59,6 +62,8 @@ public class MiniClusterResource extends ExternalResource { private JobExecutorService jobExecutorService; + private ClusterClient clusterClient; + private int numberSlots = -1; private TestEnvironment executionEnvironment; @@ -80,6 +85,10 @@ public int getNumberSlots() { return numberSlots; } + public ClusterClient getClusterClient() { + return clusterClient; + } + public TestEnvironment getTestEnvironment() { return executionEnvironment; } @@ -87,7 +96,7 @@ public TestEnvironment getTestEnvironment() { @Override public void before() throws Exception { - jobExecutorService = startJobExecutorService(miniClusterType); + startJobExecutorService(miniClusterType); numberSlots = miniClusterResourceConfiguration.getNumberSlotsPerTaskManager() * miniClusterResourceConfiguration.getNumberTaskManagers(); @@ -102,6 +111,16 @@ public void after() { TestStreamEnvironment.unsetAsContext(); TestEnvironment.unsetAsContext(); + Exception exception = null; + + try { + clusterClient.shutdown(); + } catch (Exception e) { + exception = e; + } + + clusterClient = null; + final CompletableFuture terminationFuture = jobExecutorService.closeAsync(); try { @@ -109,40 +128,43 @@ public void after() { miniClusterResourceConfiguration.getShutdownTimeout().toMilliseconds(), TimeUnit.MILLISECONDS); } catch (Exception e) { - LOG.warn("Could not properly shut down the MiniClusterResource.", e); + exception = ExceptionUtils.firstOrSuppressed(e, exception); } jobExecutorService = null; + + if (exception != null) { + LOG.warn("Could not properly shut down the MiniClusterResource.", exception); + } } - private JobExecutorService startJobExecutorService(MiniClusterType miniClusterType) throws Exception { - final JobExecutorService jobExecutorService; + private void startJobExecutorService(MiniClusterType miniClusterType) throws Exception { switch (miniClusterType) { case OLD: - jobExecutorService = startOldMiniCluster(); + startOldMiniCluster(); break; case FLIP6: - jobExecutorService = startFlip6MiniCluster(); + startFlip6MiniCluster(); break; default: throw new FlinkRuntimeException("Unknown MiniClusterType " + miniClusterType + '.'); } - - return jobExecutorService; } - private JobExecutorService startOldMiniCluster() throws Exception { + private void startOldMiniCluster() throws Exception { final Configuration configuration = new Configuration(miniClusterResourceConfiguration.getConfiguration()); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, miniClusterResourceConfiguration.getNumberTaskManagers()); configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, miniClusterResourceConfiguration.getNumberSlotsPerTaskManager()); - return TestBaseUtils.startCluster( + final LocalFlinkMiniCluster flinkMiniCluster = TestBaseUtils.startCluster( configuration, true); + + jobExecutorService = flinkMiniCluster; + clusterClient = new StandaloneClusterClient(configuration, flinkMiniCluster.highAvailabilityServices()); } - @Nonnull - private JobExecutorService startFlip6MiniCluster() throws Exception { + private void startFlip6MiniCluster() throws Exception { final Configuration configuration = miniClusterResourceConfiguration.getConfiguration(); // we need to set this since a lot of test expect this because TestBaseUtils.startCluster() @@ -165,7 +187,8 @@ private JobExecutorService startFlip6MiniCluster() throws Exception { // update the port of the rest endpoint configuration.setInteger(RestOptions.REST_PORT, miniCluster.getRestAddress().getPort()); - return miniCluster; + jobExecutorService = miniCluster; + clusterClient = new MiniClusterClient(configuration, miniCluster); } /** From 19e4f68ba9cfbf5d0f54b325db4c5d196d262d09 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Feb 2018 17:49:29 +0100 Subject: [PATCH 041/268] [FLINK-8811] [flip6] Implement MiniClusterClient#getJobStatus --- .../client/program/MiniClusterClient.java | 7 ++++++- .../flink/runtime/minicluster/MiniCluster.java | 18 ++++++++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index 5baae5b0a3e63..e99addda9e813 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.minicluster.MiniCluster; @@ -57,7 +58,7 @@ public MiniClusterClient(@Nonnull Configuration configuration, @Nonnull MiniClus } @Override - protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { + public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { if (isDetached()) { try { miniCluster.runDetached(jobGraph); @@ -119,6 +120,10 @@ public Map getAccumulators(JobID jobID, ClassLoader loader) thro throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); } + public CompletableFuture getJobStatus(JobID jobId) { + return miniCluster.getJobStatus(jobId); + } + @Override public MiniClusterClient.MiniClusterId getClusterId() { return MiniClusterId.INSTANCE; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index cbfb266499287..5b086ca55de20 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.minicluster; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.io.FileOutputFormat; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; @@ -41,6 +42,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -63,6 +65,7 @@ import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever; import org.apache.flink.util.AutoCloseableAsync; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import akka.actor.ActorSystem; import com.typesafe.config.Config; @@ -454,6 +457,21 @@ public CompletableFuture closeAsync() { } } + // ------------------------------------------------------------------------ + // Accessing jobs + // ------------------------------------------------------------------------ + + public CompletableFuture getJobStatus(JobID jobId) { + try { + return getDispatcherGateway().requestJobStatus(jobId, rpcTimeout); + } catch (LeaderRetrievalException | InterruptedException e) { + return FutureUtils.completedExceptionally( + new FlinkException( + String.format("Could not retrieve job status for job %s", jobId), + e)); + } + } + // ------------------------------------------------------------------------ // running jobs // ------------------------------------------------------------------------ From 29b34e2255c41abc1c7c4af8ab268a39df57f0ff Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 28 Feb 2018 17:54:06 +0100 Subject: [PATCH 042/268] [FLINK-8811] [flip6] Implement MiniClusterClient#cancel --- .../flink/client/program/MiniClusterClient.java | 2 +- .../flink/runtime/minicluster/MiniCluster.java | 13 ++++++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index e99addda9e813..b98e895c51558 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -82,7 +82,7 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) @Override public void cancel(JobID jobId) throws Exception { - throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + miniCluster.cancelJob(jobId); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 5b086ca55de20..2efdb03505b97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -467,7 +467,18 @@ public CompletableFuture getJobStatus(JobID jobId) { } catch (LeaderRetrievalException | InterruptedException e) { return FutureUtils.completedExceptionally( new FlinkException( - String.format("Could not retrieve job status for job %s", jobId), + String.format("Could not retrieve job status for job %s.", jobId), + e)); + } + } + + public CompletableFuture cancelJob(JobID jobId) { + try { + return getDispatcherGateway().cancelJob(jobId, rpcTimeout); + } catch (LeaderRetrievalException | InterruptedException e) { + return FutureUtils.completedExceptionally( + new FlinkException( + String.format("Could not cancel job %s.", jobId), e)); } } From 15847c4d9c3155aa9a46498d5cf3a6676e0fa08d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 17:02:29 +0100 Subject: [PATCH 043/268] [hotfix] Introduce null checks for SlotManager#suspend --- .../resourcemanager/slotmanager/SlotManager.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index f078a28fab260..ca3371945c8fd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -208,11 +208,15 @@ public void suspend() { LOG.info("Suspending the SlotManager."); // stop the timeout checks for the TaskManagers and the SlotRequests - taskManagerTimeoutCheck.cancel(false); - slotRequestTimeoutCheck.cancel(false); + if (taskManagerTimeoutCheck != null) { + taskManagerTimeoutCheck.cancel(false); + taskManagerTimeoutCheck = null; + } - taskManagerTimeoutCheck = null; - slotRequestTimeoutCheck = null; + if (slotRequestTimeoutCheck != null) { + slotRequestTimeoutCheck.cancel(false); + slotRequestTimeoutCheck = null; + } for (PendingSlotRequest pendingSlotRequest : pendingSlotRequests.values()) { cancelPendingSlotRequest(pendingSlotRequest); From 40791147efa2f3ca1c8950ed2646a3ac3a844716 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 17:05:44 +0100 Subject: [PATCH 044/268] [hotfix] Let ClusterClient only shut down own HaServices --- .../apache/flink/client/program/ClusterClient.java | 14 ++++++++++---- .../flink/client/program/MiniClusterClient.java | 2 +- .../client/program/StandaloneClusterClient.java | 4 ++-- .../flink/client/cli/CliFrontendModifyTest.java | 2 +- .../flink/client/cli/CliFrontendSavepointTest.java | 5 +++-- .../flink/client/program/ClientConnectionTest.java | 2 +- .../flink/client/program/ClusterClientTest.java | 4 ++-- .../flink/test/util/MiniClusterResource.java | 2 +- .../checkpointing/AbstractLocalRecoveryITCase.java | 12 ++++++------ .../test/example/client/JobRetrievalITCase.java | 2 +- 10 files changed, 28 insertions(+), 21 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index e2efbac523ceb..1cf2bc2847c36 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -117,6 +117,8 @@ public abstract class ClusterClient { /** Service factory for high available. */ protected final HighAvailabilityServices highAvailabilityServices; + private final boolean sharedHaServices; + /** Flag indicating whether to sysout print execution updates. */ private boolean printStatusDuringExecution = true; @@ -144,11 +146,13 @@ public abstract class ClusterClient { * @throws Exception we cannot create the high availability services */ public ClusterClient(Configuration flinkConfig) throws Exception { - this(flinkConfig, + this( + flinkConfig, HighAvailabilityServicesUtils.createHighAvailabilityServices( flinkConfig, Executors.directExecutor(), - HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION)); + HighAvailabilityServicesUtils.AddressResolution.TRY_ADDRESS_RESOLUTION), + false); } /** @@ -158,8 +162,9 @@ public ClusterClient(Configuration flinkConfig) throws Exception { * * @param flinkConfig The config used to obtain the job-manager's address, and used to configure the optimizer. * @param highAvailabilityServices HighAvailabilityServices to use for leader retrieval + * @param sharedHaServices true if the HighAvailabilityServices are shared and must not be shut down */ - public ClusterClient(Configuration flinkConfig, HighAvailabilityServices highAvailabilityServices) { + public ClusterClient(Configuration flinkConfig, HighAvailabilityServices highAvailabilityServices, boolean sharedHaServices) { this.flinkConfig = Preconditions.checkNotNull(flinkConfig); this.compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig); @@ -173,6 +178,7 @@ public ClusterClient(Configuration flinkConfig, HighAvailabilityServices highAva log); this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices); + this.sharedHaServices = sharedHaServices; } // ------------------------------------------------------------------------ @@ -265,7 +271,7 @@ public void shutdown() throws Exception { synchronized (this) { actorSystemLoader.shutdown(); - if (highAvailabilityServices != null) { + if (!sharedHaServices && highAvailabilityServices != null) { highAvailabilityServices.close(); } } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index b98e895c51558..aca75e0c15087 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -52,7 +52,7 @@ public class MiniClusterClient extends ClusterClient> rescaleJobFuture; public TestingClusterClient(CompletableFuture> rescaleJobFuture) throws Exception { - super(new Configuration(), new TestingHighAvailabilityServices()); + super(new Configuration(), new TestingHighAvailabilityServices(), false); this.rescaleJobFuture = rescaleJobFuture; } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java index d7303447da72e..f4c66eb08c204 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java @@ -138,7 +138,8 @@ public void testTriggerSavepointFailureIllegalJobID() throws Exception { try { CliFrontend frontend = new MockedCliFrontend(new StandaloneClusterClient( new Configuration(), - new TestingHighAvailabilityServices())); + new TestingHighAvailabilityServices(), + false)); String[] parameters = { "invalid job id" }; try { @@ -288,7 +289,7 @@ private static final class DisposeSavepointClusterClient extends StandaloneClust private final BiFunction> disposeSavepointFunction; DisposeSavepointClusterClient(BiFunction> disposeSavepointFunction) throws Exception { - super(new Configuration(), new TestingHighAvailabilityServices()); + super(new Configuration(), new TestingHighAvailabilityServices(), false); this.disposeSavepointFunction = Preconditions.checkNotNull(disposeSavepointFunction); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java index 1dd4787c12d2a..2b8abb1f8ff48 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientConnectionTest.java @@ -140,7 +140,7 @@ public void testJobManagerRetrievalWithHAServices() throws Exception { highAvailabilityServices.setJobMasterLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID, settableLeaderRetrievalService); - StandaloneClusterClient client = new StandaloneClusterClient(configuration, highAvailabilityServices); + StandaloneClusterClient client = new StandaloneClusterClient(configuration, highAvailabilityServices, true); ActorGateway gateway = client.getJobManagerGateway(); diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java index e2eb88d20d4b1..f30fd192e8388 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java @@ -71,7 +71,7 @@ public void testClusterClientShutdown() throws Exception { Configuration config = new Configuration(); HighAvailabilityServices highAvailabilityServices = mock(HighAvailabilityServices.class); - StandaloneClusterClient clusterClient = new StandaloneClusterClient(config, highAvailabilityServices); + StandaloneClusterClient clusterClient = new StandaloneClusterClient(config, highAvailabilityServices, false); clusterClient.shutdown(); @@ -333,7 +333,7 @@ private static class TestClusterClient extends StandaloneClusterClient { private final ActorGateway jobmanagerGateway; TestClusterClient(Configuration config, ActorGateway jobmanagerGateway) throws Exception { - super(config, new TestingHighAvailabilityServices()); + super(config, new TestingHighAvailabilityServices(), false); this.jobmanagerGateway = jobmanagerGateway; } diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java index a1ce64737571a..954b06f65a9ca 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java @@ -161,7 +161,7 @@ private void startOldMiniCluster() throws Exception { true); jobExecutorService = flinkMiniCluster; - clusterClient = new StandaloneClusterClient(configuration, flinkMiniCluster.highAvailabilityServices()); + clusterClient = new StandaloneClusterClient(configuration, flinkMiniCluster.highAvailabilityServices(), true); } private void startFlip6MiniCluster() throws Exception { diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractLocalRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractLocalRecoveryITCase.java index a02e902ab2bfd..13040c965549f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractLocalRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractLocalRecoveryITCase.java @@ -78,20 +78,20 @@ protected Configuration createClusterConfig() throws IOException { private void executeTest(AbstractEventTimeWindowCheckpointingITCase delegate) throws Exception { delegate.name = testName; try { - delegate.miniClusterResource.before(); + delegate.setupTestCluster(); try { delegate.testTumblingTimeWindow(); - delegate.miniClusterResource.after(); + delegate.stopTestCluster(); } catch (Exception e) { - delegate.miniClusterResource.after(); + delegate.stopTestCluster(); } - delegate.miniClusterResource.before(); + delegate.setupTestCluster(); try { delegate.testSlidingTimeWindow(); - delegate.miniClusterResource.after(); + delegate.stopTestCluster(); } catch (Exception e) { - delegate.miniClusterResource.after(); + delegate.stopTestCluster(); } } finally { delegate.tempFolder.delete(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java index 221f3fa145385..d34b6c337a06e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java @@ -79,7 +79,7 @@ public void testJobRetrieval() throws Exception { final JobGraph jobGraph = new JobGraph(jobID, "testjob", imalock); - final ClusterClient client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices()); + final ClusterClient client = new StandaloneClusterClient(cluster.configuration(), cluster.highAvailabilityServices(), true); // acquire the lock to make sure that the job cannot complete until the job client // has been attached in resumingThread From 2b693502318dd3bfdae352b6ae4f4be6e4443bbc Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 17:06:27 +0100 Subject: [PATCH 045/268] [hotfix] Unregister job from JobManagerRunner before completing the result future --- .../org/apache/flink/runtime/jobmaster/JobManagerRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index 80aa673c86a70..8b64f0ddf57e9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -265,8 +265,8 @@ public CompletableFuture closeAsync() { @Override public void jobReachedGloballyTerminalState(ArchivedExecutionGraph executionGraph) { // complete the result future with the terminal execution graph - resultFuture.complete(executionGraph); unregisterJobFromHighAvailability(); + resultFuture.complete(executionGraph); } /** From f0d4b3c98f71fa15b99e55634bc4d067b48149fc Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 17:13:44 +0100 Subject: [PATCH 046/268] [hotfix] Close ResourceManager LeaderRetrievalService in TaskExecutor --- .../flink/runtime/taskexecutor/TaskExecutor.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 927bd117f1601..956a319732e72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -56,6 +56,7 @@ import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.StackTraceSampleResponse; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; @@ -176,6 +177,8 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final JobLeaderService jobLeaderService; + private final LeaderRetrievalService resourceManagerLeaderRetriever; + // ------------------------------------------------------------------------ private final HardwareDescription hardwareDescription; @@ -207,6 +210,7 @@ public TaskExecutor( this.taskManagerLocation = taskExecutorServices.getTaskManagerLocation(); this.localStateStoresManager = taskExecutorServices.getTaskManagerStateStore(); this.networkEnvironment = taskExecutorServices.getNetworkEnvironment(); + this.resourceManagerLeaderRetriever = haServices.getResourceManagerLeaderRetriever(); this.jobManagerConnections = new HashMap<>(4); @@ -238,7 +242,7 @@ public void start() throws Exception { // start by connecting to the ResourceManager try { - haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener()); + resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener()); } catch (Exception e) { onFatalError(e); } @@ -275,6 +279,12 @@ public CompletableFuture postStop() { resourceManagerHeartbeatManager.stop(); + try { + resourceManagerLeaderRetriever.stop(); + } catch (Exception e) { + throwable = ExceptionUtils.firstOrSuppressed(e, throwable); + } + try { taskExecutorServices.shutDown(); } catch (Throwable t) { From ea920efc64254b16322bb1d13b5ab1c8633cbda7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 18:21:19 +0100 Subject: [PATCH 047/268] [hotfix] Correct shutdown order of RestClusterClient --- .../client/program/rest/RestClusterClient.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 98332968004bc..976f2a4db31ca 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -192,13 +192,6 @@ private void startLeaderRetrievers() throws Exception { @Override public void shutdown() { - try { - // we only call this for legacy reasons to shutdown components that are started in the ClusterClient constructor - super.shutdown(); - } catch (Exception e) { - log.error("An error occurred during the client shutdown.", e); - } - ExecutorUtils.gracefulShutdown(restClusterClientConfiguration.getRetryDelay(), TimeUnit.MILLISECONDS, retryExecutorService); this.restClient.shutdown(Time.seconds(5)); @@ -215,6 +208,13 @@ public void shutdown() { } catch (Exception e) { log.error("An error occurred during stopping the dispatcherLeaderRetriever", e); } + + try { + // we only call this for legacy reasons to shutdown components that are started in the ClusterClient constructor + super.shutdown(); + } catch (Exception e) { + log.error("An error occurred during the client shutdown.", e); + } } @Override From 8485912f807b8bc43c46fe785ec16a56d9253b5a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 19:03:35 +0100 Subject: [PATCH 048/268] [hotfix] Let AbstractEventTimeWindowCheckpointingITCase shutdown ZooKeeper after MiniCluster --- ...tractEventTimeWindowCheckpointingITCase.java | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java index f37ba0d38053e..61baefa05d8ad 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/AbstractEventTimeWindowCheckpointingITCase.java @@ -51,6 +51,7 @@ import org.apache.curator.test.TestingServer; import org.junit.After; +import org.junit.Before; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -87,6 +88,8 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog private TestingServer zkServer; + public MiniClusterResource miniClusterResource; + @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); @@ -95,9 +98,6 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog private AbstractStateBackend stateBackend; - @Rule - public final MiniClusterResource miniClusterResource = getMiniClusterResource(); - enum StateBackendEnum { MEM, FILE, ROCKSDB_FULLY_ASYNC, ROCKSDB_INCREMENTAL, ROCKSDB_INCREMENTAL_ZK, MEM_ASYNC, FILE_ASYNC } @@ -201,8 +201,19 @@ protected Configuration createClusterConfig() throws IOException { return config; } + @Before + public void setupTestCluster() throws Exception { + miniClusterResource = getMiniClusterResource(); + miniClusterResource.before(); + } + @After public void stopTestCluster() throws IOException { + if (miniClusterResource != null) { + miniClusterResource.after(); + miniClusterResource = null; + } + if (zkServer != null) { zkServer.stop(); zkServer = null; From 007cf2b2ae7ac9c31e04e183fbc0dbe527399d63 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 19:04:15 +0100 Subject: [PATCH 049/268] [hotfix] Let JobLeaderService terminate leader retrieval services --- .../flink/runtime/taskexecutor/JobLeaderService.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java index 53763629079ae..500d7e4a297df 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderService.java @@ -190,7 +190,12 @@ public void addJob(final JobID jobId, final String defaultTargetAddress) throws JobLeaderService.JobManagerLeaderListener jobManagerLeaderListener = new JobManagerLeaderListener(jobId); - jobLeaderServices.put(jobId, Tuple2.of(leaderRetrievalService, jobManagerLeaderListener)); + final Tuple2 oldEntry = jobLeaderServices.put(jobId, Tuple2.of(leaderRetrievalService, jobManagerLeaderListener)); + + if (oldEntry != null) { + oldEntry.f0.stop(); + oldEntry.f1.stop(); + } leaderRetrievalService.start(jobManagerLeaderListener); } From 81aa5aceef45ecb4e07125259f749785eb66a64a Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 19:04:53 +0100 Subject: [PATCH 050/268] [hotfix] Improve logging in ZooKeeper services --- .../flink/runtime/dispatcher/Dispatcher.java | 2 ++ .../zookeeper/ZooKeeperHaServices.java | 6 ++++++ .../ZooKeeperLeaderRetrievalService.java | 7 +++++-- .../flink/runtime/minicluster/MiniCluster.java | 14 +++++++------- .../flink/runtime/taskexecutor/TaskExecutor.java | 5 +++-- 5 files changed, 23 insertions(+), 11 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index bb60277f32503..7a11cf0878418 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -194,6 +194,8 @@ public CompletableFuture postStop() { if (exception != null) { throw exception; + } else { + log.info("Stopped dispatcher {}.", getAddress()); } }); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java index 6d5c721daadb6..3882479ce95f0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServices.java @@ -34,6 +34,8 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.curator.framework.CuratorFramework; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.concurrent.Executor; @@ -80,6 +82,8 @@ */ public class ZooKeeperHaServices implements HighAvailabilityServices { + private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperHaServices.class); + private static final String RESOURCE_MANAGER_LEADER_PATH = "/resource_manager_lock"; private static final String DISPATCHER_LEADER_PATH = "/dispatcher_lock"; @@ -211,6 +215,8 @@ public void close() throws Exception { @Override public void closeAndCleanupAllData() throws Exception { + LOG.info("Close and clean up all data for ZooKeeperHaServices."); + Throwable exception = null; try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java index 00ba66c4ebe16..fd0ea82b6e045 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalService.java @@ -55,6 +55,8 @@ public class ZooKeeperLeaderRetrievalService implements LeaderRetrievalService, /** Curator recipe to watch changes of a specific ZooKeeper node. */ private final NodeCache cache; + private final String retrievalPath; + /** Listener which will be notified about leader changes. */ private volatile LeaderRetrievalListener leaderListener; @@ -80,6 +82,7 @@ public void stateChanged(CuratorFramework client, ConnectionState newState) { public ZooKeeperLeaderRetrievalService(CuratorFramework client, String retrievalPath) { this.client = Preconditions.checkNotNull(client, "CuratorFramework client"); this.cache = new NodeCache(client, retrievalPath); + this.retrievalPath = Preconditions.checkNotNull(retrievalPath); this.leaderListener = null; this.lastLeaderAddress = null; @@ -94,7 +97,7 @@ public void start(LeaderRetrievalListener listener) throws Exception { Preconditions.checkState(leaderListener == null, "ZooKeeperLeaderRetrievalService can " + "only be started once."); - LOG.info("Starting ZooKeeperLeaderRetrievalService."); + LOG.info("Starting ZooKeeperLeaderRetrievalService {}.", retrievalPath); synchronized (lock) { leaderListener = listener; @@ -111,7 +114,7 @@ public void start(LeaderRetrievalListener listener) throws Exception { @Override public void stop() throws Exception { - LOG.info("Stopping ZooKeeperLeaderRetrievalService."); + LOG.info("Stopping ZooKeeperLeaderRetrievalService {}.", retrievalPath); synchronized (lock) { if (!running) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 2efdb03505b97..6b5f9b50aecca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -401,13 +401,6 @@ public CompletableFuture closeAsync() { final int numComponents = 2 + miniClusterConfiguration.getNumTaskManagers(); final Collection> componentTerminationFutures = new ArrayList<>(numComponents); - componentTerminationFutures.add(shutDownDispatcher()); - - if (resourceManagerRunner != null) { - componentTerminationFutures.add(resourceManagerRunner.closeAsync()); - resourceManagerRunner = null; - } - if (taskManagers != null) { for (TaskExecutor tm : taskManagers) { if (tm != null) { @@ -418,6 +411,13 @@ public CompletableFuture closeAsync() { taskManagers = null; } + componentTerminationFutures.add(shutDownDispatcher()); + + if (resourceManagerRunner != null) { + componentTerminationFutures.add(resourceManagerRunner.closeAsync()); + resourceManagerRunner = null; + } + final FutureUtils.ConjunctFuture componentsTerminationFuture = FutureUtils.completeAll(componentTerminationFutures); final CompletableFuture metricRegistryTerminationFuture = FutureUtils.runAfterwards( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 956a319732e72..cab686af59aa9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -259,7 +259,7 @@ public void start() throws Exception { */ @Override public CompletableFuture postStop() { - log.info("Stopping TaskManager {}.", getAddress()); + log.info("Stopping TaskExecutor {}.", getAddress()); Throwable throwable = null; @@ -294,6 +294,7 @@ public CompletableFuture postStop() { if (throwable != null) { return FutureUtils.completedExceptionally(new FlinkException("Error while shutting the TaskExecutor down.", throwable)); } else { + log.info("Stopped TaskExecutor {}.", getAddress()); return CompletableFuture.completedFuture(null); } } @@ -1333,7 +1334,7 @@ public ResourceID getResourceID() { */ void onFatalError(final Throwable t) { try { - log.error("Fatal error occurred in TaskExecutor.", t); + log.error("Fatal error occurred in TaskExecutor {}.", getAddress(), t); } catch (Throwable ignored) {} // The fatal error handler implementation should make sure that this call is non-blocking From 347ec3848ec603ac452e394a5211cf888db6663f Mon Sep 17 00:00:00 2001 From: Xpray Date: Fri, 2 Mar 2018 12:11:45 +0800 Subject: [PATCH 051/268] [FLINK-8821] [table] Fix non-terminating decimal error This closes #5608. --- .../apache/flink/table/api/TableConfig.scala | 21 +++++++ .../flink/table/codegen/CodeGenerator.scala | 18 +++--- .../table/codegen/calls/ScalarOperators.scala | 29 ++++++++-- .../aggfunctions/AvgAggFunction.scala | 7 ++- .../plan/nodes/dataset/DataSetAggregate.scala | 3 +- .../dataset/DataSetWindowAggregate.scala | 55 ++++++++++++------ .../datastream/DataStreamGroupAggregate.scala | 1 + .../DataStreamGroupWindowAggregate.scala | 6 +- .../datastream/DataStreamOverAggregate.scala | 12 +++- .../runtime/aggregate/AggregateUtil.scala | 57 +++++++++++++------ .../table/expressions/DecimalTypeTest.scala | 11 +++- .../aggfunctions/AvgFunctionTest.scala | 12 +++- .../runtime/batch/table/CalcITCase.scala | 8 ++- 13 files changed, 176 insertions(+), 64 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala index 6448657c5111f..c78a022bec44a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.api import _root_.java.util.TimeZone +import _root_.java.math.MathContext import org.apache.flink.table.calcite.CalciteConfig @@ -41,6 +42,12 @@ class TableConfig { */ private var calciteConfig = CalciteConfig.DEFAULT + /** + * Defines the default context for decimal division calculation. + * We use Scala's default MathContext.DECIMAL128. + */ + private var decimalContext = MathContext.DECIMAL128 + /** * Sets the timezone for date/time/timestamp conversions. */ @@ -78,6 +85,20 @@ class TableConfig { def setCalciteConfig(calciteConfig: CalciteConfig): Unit = { this.calciteConfig = calciteConfig } + + /** + * Returns the default context for decimal division calculation. + * [[_root_.java.math.MathContext#DECIMAL128]] by default. + */ + def getDecimalContext: MathContext = decimalContext + + /** + * Sets the default context for decimal division calculation. + * [[_root_.java.math.MathContext#DECIMAL128]] by default. + */ + def setDecimalContext(mathContext: MathContext): Unit = { + this.decimalContext = mathContext + } } object TableConfig { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index 756a8288dcaff..e4064d6f1acd8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -742,56 +742,56 @@ abstract class CodeGenerator( val right = operands(1) requireNumeric(left) requireNumeric(right) - generateArithmeticOperator("+", nullCheck, resultType, left, right) + generateArithmeticOperator("+", nullCheck, resultType, left, right, config) case PLUS | DATETIME_PLUS if isTemporal(resultType) => val left = operands.head val right = operands(1) requireTemporal(left) requireTemporal(right) - generateTemporalPlusMinus(plus = true, nullCheck, left, right) + generateTemporalPlusMinus(plus = true, nullCheck, left, right, config) case MINUS if isNumeric(resultType) => val left = operands.head val right = operands(1) requireNumeric(left) requireNumeric(right) - generateArithmeticOperator("-", nullCheck, resultType, left, right) + generateArithmeticOperator("-", nullCheck, resultType, left, right, config) case MINUS | MINUS_DATE if isTemporal(resultType) => val left = operands.head val right = operands(1) requireTemporal(left) requireTemporal(right) - generateTemporalPlusMinus(plus = false, nullCheck, left, right) + generateTemporalPlusMinus(plus = false, nullCheck, left, right, config) case MULTIPLY if isNumeric(resultType) => val left = operands.head val right = operands(1) requireNumeric(left) requireNumeric(right) - generateArithmeticOperator("*", nullCheck, resultType, left, right) + generateArithmeticOperator("*", nullCheck, resultType, left, right, config) case MULTIPLY if isTimeInterval(resultType) => val left = operands.head val right = operands(1) requireTimeInterval(left) requireNumeric(right) - generateArithmeticOperator("*", nullCheck, resultType, left, right) + generateArithmeticOperator("*", nullCheck, resultType, left, right, config) case DIVIDE | DIVIDE_INTEGER if isNumeric(resultType) => val left = operands.head val right = operands(1) requireNumeric(left) requireNumeric(right) - generateArithmeticOperator("/", nullCheck, resultType, left, right) + generateArithmeticOperator("/", nullCheck, resultType, left, right, config) case MOD if isNumeric(resultType) => val left = operands.head val right = operands(1) requireNumeric(left) requireNumeric(right) - generateArithmeticOperator("%", nullCheck, resultType, left, right) + generateArithmeticOperator("%", nullCheck, resultType, left, right, config) case UNARY_MINUS if isNumeric(resultType) => val operand = operands.head @@ -922,7 +922,7 @@ abstract class CodeGenerator( val left = operands.head val right = operands(1) requireString(left) - generateArithmeticOperator("+", nullCheck, resultType, left, right) + generateArithmeticOperator("+", nullCheck, resultType, left, right, config) // rows case ROW => diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala index a261b3d8ebbd4..57f1618a2e363 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala @@ -17,12 +17,15 @@ */ package org.apache.flink.table.codegen.calls +import java.math.MathContext + import org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY import org.apache.calcite.avatica.util.{DateTimeUtils, TimeUnitRange} import org.apache.calcite.util.BuiltInMethod import org.apache.flink.api.common.typeinfo.BasicTypeInfo._ import org.apache.flink.api.common.typeinfo._ import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo} +import org.apache.flink.table.api.TableConfig import org.apache.flink.table.codegen.CodeGenUtils._ import org.apache.flink.table.codegen.calls.CallGenerator.generateCallIfArgsNotNull import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, GeneratedExpression} @@ -46,7 +49,8 @@ object ScalarOperators { nullCheck: Boolean, resultType: TypeInformation[_], left: GeneratedExpression, - right: GeneratedExpression): GeneratedExpression = { + right: GeneratedExpression, + config: TableConfig): GeneratedExpression = { val leftCasting = operator match { case "%" => @@ -68,7 +72,15 @@ object ScalarOperators { generateOperatorIfNotNull(nullCheck, resultType, left, right) { (leftTerm, rightTerm) => if (isDecimal(resultType)) { - s"${leftCasting(leftTerm)}.${arithOpToDecMethod(operator)}(${rightCasting(rightTerm)})" + val decMethod = arithOpToDecMethod(operator) + operator match { + // include math context for decimal division + case "/" => + val mathContext = mathContextToString(config.getDecimalContext) + s"${leftCasting(leftTerm)}.$decMethod(${rightCasting(rightTerm)}, $mathContext)" + case _ => + s"${leftCasting(leftTerm)}.$decMethod(${rightCasting(rightTerm)})" + } } else { s"($resultTypeTerm) (${leftCasting(leftTerm)} $operator ${rightCasting(rightTerm)})" } @@ -814,14 +826,15 @@ object ScalarOperators { plus: Boolean, nullCheck: Boolean, left: GeneratedExpression, - right: GeneratedExpression) + right: GeneratedExpression, + config: TableConfig) : GeneratedExpression = { val op = if (plus) "+" else "-" (left.resultType, right.resultType) match { case (l: TimeIntervalTypeInfo[_], r: TimeIntervalTypeInfo[_]) if l == r => - generateArithmeticOperator(op, nullCheck, l, left, right) + generateArithmeticOperator(op, nullCheck, l, left, right, config) case (SqlTimeTypeInfo.DATE, TimeIntervalTypeInfo.INTERVAL_MILLIS) => generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.DATE, left, right) { @@ -1290,6 +1303,14 @@ object ScalarOperators { case _ => throw new CodeGenException(s"Unsupported decimal arithmetic operator: '$operator'") } + private def mathContextToString(mathContext: MathContext): String = mathContext match { + case MathContext.DECIMAL32 => "java.math.MathContext.DECIMAL32" + case MathContext.DECIMAL64 => "java.math.MathContext.DECIMAL64" + case MathContext.DECIMAL128 => "java.math.MathContext.DECIMAL128" + case MathContext.UNLIMITED => "java.math.MathContext.UNLIMITED" + case _ => s"""new java.math.MathContext("$mathContext")""" + } + private def numericCasting( operandType: TypeInformation[_], resultType: TypeInformation[_]) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.scala index b651c424dc766..26621b71961a3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/aggfunctions/AvgAggFunction.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.table.functions.aggfunctions -import java.math.{BigDecimal, BigInteger} +import java.math.{BigDecimal, BigInteger, MathContext} import java.lang.{Iterable => JIterable} import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} @@ -295,7 +295,8 @@ class DecimalAvgAccumulator extends JTuple2[BigDecimal, Long] { /** * Base class for built-in Big Decimal Avg aggregate function */ -class DecimalAvgAggFunction extends AggregateFunction[BigDecimal, DecimalAvgAccumulator] { +class DecimalAvgAggFunction(context: MathContext) + extends AggregateFunction[BigDecimal, DecimalAvgAccumulator] { override def createAccumulator(): DecimalAvgAccumulator = { new DecimalAvgAccumulator @@ -321,7 +322,7 @@ class DecimalAvgAggFunction extends AggregateFunction[BigDecimal, DecimalAvgAccu if (acc.f1 == 0) { null.asInstanceOf[BigDecimal] } else { - acc.f0.divide(BigDecimal.valueOf(acc.f1)) + acc.f0.divide(BigDecimal.valueOf(acc.f1), context) } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala index 7dd307b064281..07dcf798621d3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala @@ -110,7 +110,8 @@ class DataSetAggregate( input.getRowType, inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, rowRelDataType, - grouping) + grouping, + tableEnv.getConfig) val aggString = aggregationToString(inputType, grouping, getRowType, namedAggregates, Nil) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala index 745c4ed708a41..53748f5c71cd0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetWindowAggregate.scala @@ -25,7 +25,7 @@ import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} import org.apache.flink.api.common.operators.Order import org.apache.flink.api.java.DataSet import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, RowTypeInfo} -import org.apache.flink.table.api.{BatchQueryConfig, BatchTableEnvironment} +import org.apache.flink.table.api.{BatchQueryConfig, BatchTableEnvironment, TableConfig} import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.AggregationCodeGenerator @@ -127,11 +127,12 @@ class DataSetWindowAggregate( generator, inputDS, isTimeIntervalLiteral(size), - caseSensitive) + caseSensitive, + tableEnv.getConfig) case SessionGroupWindow(_, timeField, gap) if isTimePoint(timeField.resultType) || isLong(timeField.resultType) => - createEventTimeSessionWindowDataSet(generator, inputDS, caseSensitive) + createEventTimeSessionWindowDataSet(generator, inputDS, caseSensitive, tableEnv.getConfig) case SlidingGroupWindow(_, timeField, size, slide) if isTimePoint(timeField.resultType) || isLong(timeField.resultType) => @@ -141,7 +142,8 @@ class DataSetWindowAggregate( isTimeIntervalLiteral(size), asLong(size), asLong(slide), - caseSensitive) + caseSensitive, + tableEnv.getConfig) case _ => throw new UnsupportedOperationException( @@ -153,7 +155,8 @@ class DataSetWindowAggregate( generator: AggregationCodeGenerator, inputDS: DataSet[Row], isTimeWindow: Boolean, - isParserCaseSensitive: Boolean): DataSet[Row] = { + isParserCaseSensitive: Boolean, + tableConfig: TableConfig): DataSet[Row] = { val input = inputNode.asInstanceOf[DataSetRel] @@ -164,7 +167,8 @@ class DataSetWindowAggregate( grouping, input.getRowType, inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, - isParserCaseSensitive) + isParserCaseSensitive, + tableConfig) val groupReduceFunction = createDataSetWindowAggregationGroupReduceFunction( generator, window, @@ -173,7 +177,8 @@ class DataSetWindowAggregate( inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, getRowType, grouping, - namedProperties) + namedProperties, + tableConfig) val mappedInput = inputDS .map(mapFunction) @@ -215,7 +220,8 @@ class DataSetWindowAggregate( private[this] def createEventTimeSessionWindowDataSet( generator: AggregationCodeGenerator, inputDS: DataSet[Row], - isParserCaseSensitive: Boolean): DataSet[Row] = { + isParserCaseSensitive: Boolean, + tableConfig: TableConfig): DataSet[Row] = { val input = inputNode.asInstanceOf[DataSetRel] @@ -230,7 +236,8 @@ class DataSetWindowAggregate( grouping, input.getRowType, inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, - isParserCaseSensitive) + isParserCaseSensitive, + tableConfig) val mappedInput = inputDS.map(mapFunction).name(prepareOperatorName) @@ -243,7 +250,8 @@ class DataSetWindowAggregate( if (doAllSupportPartialMerge( namedAggregates.map(_.getKey), inputType, - grouping.length)) { + grouping.length, + tableConfig)) { // gets the window-start and window-end position in the intermediate result. val windowStartPos = rowTimeFieldPos @@ -257,7 +265,8 @@ class DataSetWindowAggregate( namedAggregates, input.getRowType, inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, - grouping) + grouping, + tableConfig) // create groupReduceFunction for calculating the aggregations val groupReduceFunction = createDataSetWindowAggregationGroupReduceFunction( @@ -269,6 +278,7 @@ class DataSetWindowAggregate( rowRelDataType, grouping, namedProperties, + tableConfig, isInputCombined = true) mappedInput @@ -289,7 +299,8 @@ class DataSetWindowAggregate( namedAggregates, input.getRowType, inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, - grouping) + grouping, + tableConfig) // create groupReduceFunction for calculating the aggregations val groupReduceFunction = createDataSetWindowAggregationGroupReduceFunction( @@ -301,6 +312,7 @@ class DataSetWindowAggregate( rowRelDataType, grouping, namedProperties, + tableConfig, isInputCombined = true) mappedInput.sortPartition(rowTimeFieldPos, Order.ASCENDING) @@ -326,7 +338,8 @@ class DataSetWindowAggregate( inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, rowRelDataType, grouping, - namedProperties) + namedProperties, + tableConfig) mappedInput.groupBy(groupingKeys: _*) .sortGroup(rowTimeFieldPos, Order.ASCENDING) @@ -343,7 +356,8 @@ class DataSetWindowAggregate( inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, rowRelDataType, grouping, - namedProperties) + namedProperties, + tableConfig) mappedInput.sortPartition(rowTimeFieldPos, Order.ASCENDING).setParallelism(1) .reduceGroup(groupReduceFunction) @@ -360,7 +374,8 @@ class DataSetWindowAggregate( isTimeWindow: Boolean, size: Long, slide: Long, - isParserCaseSensitive: Boolean) + isParserCaseSensitive: Boolean, + tableConfig: TableConfig) : DataSet[Row] = { val input = inputNode.asInstanceOf[DataSetRel] @@ -374,7 +389,8 @@ class DataSetWindowAggregate( grouping, input.getRowType, inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, - isParserCaseSensitive) + isParserCaseSensitive, + tableConfig) val mappedDataSet = inputDS .map(mapFunction) @@ -389,7 +405,8 @@ class DataSetWindowAggregate( val isPartial = doAllSupportPartialMerge( namedAggregates.map(_.getKey), inputType, - grouping.length) + grouping.length, + tableConfig) // only pre-tumble if it is worth it val isLittleTumblingSize = determineLargestTumblingSize(size, slide) <= 1 @@ -411,7 +428,8 @@ class DataSetWindowAggregate( grouping, input.getRowType, inputDS.getType.asInstanceOf[RowTypeInfo].getFieldTypes, - isParserCaseSensitive) + isParserCaseSensitive, + tableConfig) mappedDataSet.asInstanceOf[DataSet[Row]] .groupBy(groupingKeysAndAlignedRowtime: _*) @@ -451,6 +469,7 @@ class DataSetWindowAggregate( rowRelDataType, grouping, namedProperties, + tableConfig, isInputCombined = false) // gets the window-start position in the intermediate result. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala index 71de57c942c5c..5f4b186ca410e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupAggregate.scala @@ -138,6 +138,7 @@ class DataStreamGroupAggregate( inputSchema.fieldTypeInfos, groupings, queryConfig, + tableEnv.getConfig, DataStreamRetractionRules.isAccRetract(this), DataStreamRetractionRules.isAccRetract(getInput)) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala index d527dc8b94f6c..0a014b6bd2db7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamGroupWindowAggregate.scala @@ -207,7 +207,8 @@ class DataStreamGroupWindowAggregate( inputSchema.fieldTypeInfos, schema.relDataType, grouping, - needMerge) + needMerge, + tableEnv.getConfig) windowedStream .aggregate(aggFunction, windowFunction, accumulatorRowType, aggResultRowType, outRowType) @@ -232,7 +233,8 @@ class DataStreamGroupWindowAggregate( inputSchema.fieldTypeInfos, schema.relDataType, Array[Int](), - needMerge) + needMerge, + tableEnv.getConfig) windowedStream .aggregate(aggFunction, windowFunction, accumulatorRowType, aggResultRowType, outRowType) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala index 635c7bc2d0322..c1693d98515de 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala @@ -28,7 +28,7 @@ import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} import org.apache.calcite.rex.RexLiteral import org.apache.flink.api.java.functions.NullByteKeySelector import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableConfig, TableException} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.AggregationCodeGenerator import org.apache.flink.table.plan.nodes.OverAggregate @@ -172,6 +172,7 @@ class DataStreamOverAggregate( // unbounded OVER window createUnboundedAndCurrentRowOverWindow( queryConfig, + tableEnv.getConfig, generator, inputDS, rowTimeIdx, @@ -188,7 +189,8 @@ class DataStreamOverAggregate( inputDS, rowTimeIdx, aggregateInputType, - isRowsClause = overWindow.isRows) + isRowsClause = overWindow.isRows, + tableEnv.getConfig) } else { throw new TableException("OVER RANGE FOLLOWING windows are not supported yet.") } @@ -196,6 +198,7 @@ class DataStreamOverAggregate( def createUnboundedAndCurrentRowOverWindow( queryConfig: StreamQueryConfig, + tableConfig: TableConfig, generator: AggregationCodeGenerator, inputDS: DataStream[CRow], rowTimeIdx: Option[Int], @@ -219,6 +222,7 @@ class DataStreamOverAggregate( inputSchema.typeInfo, inputSchema.fieldTypeInfos, queryConfig, + tableConfig, rowTimeIdx, partitionKeys.nonEmpty, isRowsClause) @@ -249,7 +253,8 @@ class DataStreamOverAggregate( inputDS: DataStream[CRow], rowTimeIdx: Option[Int], aggregateInputType: RelDataType, - isRowsClause: Boolean): DataStream[CRow] = { + isRowsClause: Boolean, + tableConfig: TableConfig): DataStream[CRow] = { val overWindow: Group = logicWindow.groups.get(0) @@ -272,6 +277,7 @@ class DataStreamOverAggregate( inputSchema.fieldTypeInfos, precedingOffset, queryConfig, + tableConfig, isRowsClause, rowTimeIdx ) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala index 361a87e2147ad..df9b1c5520467 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala @@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.streaming.api.functions.windowing.{AllWindowFunction, WindowFunction} import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow} import org.apache.flink.table.api.dataview.DataViewSpec -import org.apache.flink.table.api.{StreamQueryConfig, TableException} +import org.apache.flink.table.api.{StreamQueryConfig, TableConfig, TableException} import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.AggregationCodeGenerator @@ -78,6 +78,7 @@ object AggregateUtil { inputTypeInfo: TypeInformation[Row], inputFieldTypeInfo: Seq[TypeInformation[_]], queryConfig: StreamQueryConfig, + tableConfig: TableConfig, rowTimeIdx: Option[Int], isPartitioned: Boolean, isRowsClause: Boolean) @@ -88,6 +89,7 @@ object AggregateUtil { namedAggregates.map(_.getKey), aggregateInputType, needRetraction = false, + tableConfig, isStateBackedDataViews = true) val aggregationStateType: RowTypeInfo = new RowTypeInfo(accTypes: _*) @@ -159,6 +161,7 @@ object AggregateUtil { inputFieldTypes: Seq[TypeInformation[_]], groupings: Array[Int], queryConfig: StreamQueryConfig, + tableConfig: TableConfig, generateRetraction: Boolean, consumeRetraction: Boolean): ProcessFunction[CRow, CRow] = { @@ -167,6 +170,7 @@ object AggregateUtil { namedAggregates.map(_.getKey), inputRowType, consumeRetraction, + tableConfig, isStateBackedDataViews = true) val aggMapping = aggregates.indices.map(_ + groupings.length).toArray @@ -223,6 +227,7 @@ object AggregateUtil { inputFieldTypeInfo: Seq[TypeInformation[_]], precedingOffset: Long, queryConfig: StreamQueryConfig, + tableConfig: TableConfig, isRowsClause: Boolean, rowTimeIdx: Option[Int]) : ProcessFunction[CRow, CRow] = { @@ -233,6 +238,7 @@ object AggregateUtil { namedAggregates.map(_.getKey), aggregateInputType, needRetract, + tableConfig, isStateBackedDataViews = true) val aggregationStateType: RowTypeInfo = new RowTypeInfo(accTypes: _*) @@ -325,14 +331,16 @@ object AggregateUtil { groupings: Array[Int], inputType: RelDataType, inputFieldTypeInfo: Seq[TypeInformation[_]], - isParserCaseSensitive: Boolean) + isParserCaseSensitive: Boolean, + tableConfig: TableConfig) : MapFunction[Row, Row] = { val needRetract = false val (aggFieldIndexes, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), inputType, - needRetract) + needRetract, + tableConfig) val mapReturnType: RowTypeInfo = createRowTypeForKeysAndAggregates( @@ -430,14 +438,16 @@ object AggregateUtil { groupings: Array[Int], physicalInputRowType: RelDataType, physicalInputTypes: Seq[TypeInformation[_]], - isParserCaseSensitive: Boolean) + isParserCaseSensitive: Boolean, + tableConfig: TableConfig) : RichGroupReduceFunction[Row, Row] = { val needRetract = false val (aggFieldIndexes, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), physicalInputRowType, - needRetract) + needRetract, + tableConfig) val returnType: RowTypeInfo = createRowTypeForKeysAndAggregates( groupings, @@ -543,6 +553,7 @@ object AggregateUtil { outputType: RelDataType, groupings: Array[Int], properties: Seq[NamedWindowProperty], + tableConfig: TableConfig, isInputCombined: Boolean = false) : RichGroupReduceFunction[Row, Row] = { @@ -550,7 +561,8 @@ object AggregateUtil { val (aggFieldIndexes, aggregates, _, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), physicalInputRowType, - needRetract) + needRetract, + tableConfig) val aggMapping = aggregates.indices.toArray.map(_ + groupings.length) @@ -695,13 +707,15 @@ object AggregateUtil { namedAggregates: Seq[CalcitePair[AggregateCall, String]], physicalInputRowType: RelDataType, physicalInputTypes: Seq[TypeInformation[_]], - groupings: Array[Int]): MapPartitionFunction[Row, Row] = { + groupings: Array[Int], + tableConfig: TableConfig): MapPartitionFunction[Row, Row] = { val needRetract = false val (aggFieldIndexes, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), physicalInputRowType, - needRetract) + needRetract, + tableConfig) val aggMapping = aggregates.indices.map(_ + groupings.length).toArray @@ -767,14 +781,16 @@ object AggregateUtil { namedAggregates: Seq[CalcitePair[AggregateCall, String]], physicalInputRowType: RelDataType, physicalInputTypes: Seq[TypeInformation[_]], - groupings: Array[Int]) + groupings: Array[Int], + tableConfig: TableConfig) : GroupCombineFunction[Row, Row] = { val needRetract = false val (aggFieldIndexes, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), physicalInputRowType, - needRetract) + needRetract, + tableConfig) val aggMapping = aggregates.indices.map(_ + groupings.length).toArray @@ -831,7 +847,8 @@ object AggregateUtil { inputType: RelDataType, inputFieldTypeInfo: Seq[TypeInformation[_]], outputType: RelDataType, - groupings: Array[Int]): ( + groupings: Array[Int], + tableConfig: TableConfig): ( Option[DataSetPreAggFunction], Option[TypeInformation[Row]], Either[DataSetAggFunction, DataSetFinalAggFunction]) = { @@ -840,7 +857,8 @@ object AggregateUtil { val (aggInFields, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), inputType, - needRetract) + needRetract, + tableConfig) val (gkeyOutMapping, aggOutMapping) = getOutputMappings( namedAggregates, @@ -992,7 +1010,8 @@ object AggregateUtil { inputFieldTypeInfo: Seq[TypeInformation[_]], outputType: RelDataType, groupingKeys: Array[Int], - needMerge: Boolean) + needMerge: Boolean, + tableConfig: TableConfig) : (DataStreamAggFunction[CRow, Row, Row], RowTypeInfo, RowTypeInfo) = { val needRetract = false @@ -1000,7 +1019,8 @@ object AggregateUtil { transformToAggregateFunctions( namedAggregates.map(_.getKey), inputType, - needRetract) + needRetract, + tableConfig) val aggMapping = aggregates.indices.toArray val outputArity = aggregates.length @@ -1036,12 +1056,14 @@ object AggregateUtil { private[flink] def doAllSupportPartialMerge( aggregateCalls: Seq[AggregateCall], inputType: RelDataType, - groupKeysCount: Int): Boolean = { + groupKeysCount: Int, + tableConfig: TableConfig): Boolean = { val aggregateList = transformToAggregateFunctions( aggregateCalls, inputType, - needRetraction = false)._2 + needRetraction = false, + tableConfig)._2 doAllSupportPartialMerge(aggregateList) } @@ -1121,6 +1143,7 @@ object AggregateUtil { aggregateCalls: Seq[AggregateCall], aggregateInputType: RelDataType, needRetraction: Boolean, + tableConfig: TableConfig, isStateBackedDataViews: Boolean = false) : (Array[Array[Int]], Array[TableAggregateFunction[_, _]], @@ -1251,7 +1274,7 @@ object AggregateUtil { case DOUBLE => new DoubleAvgAggFunction case DECIMAL => - new DecimalAvgAggFunction + new DecimalAvgAggFunction(tableConfig.getDecimalContext) case sqlType: SqlTypeName => throw new TableException(s"Avg aggregate does no support type: '$sqlType'") } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala index 42f800817701d..5de6f2c5aaa24 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala @@ -233,6 +233,13 @@ class DecimalTypeTest extends ExpressionTestBase { "-f0", "-f0", "-123456789.123456789123456789") + + testAllApis( + BigDecimal("1").toExpr / BigDecimal("3"), + "1p / 3p", + "CAST('1' AS DECIMAL) / CAST('3' AS DECIMAL)", + "0.3333333333333333333333333333333333" + ) } @Test @@ -287,7 +294,7 @@ class DecimalTypeTest extends ExpressionTestBase { // ---------------------------------------------------------------------------------------------- - def testData = { + def testData: Row = { val testData = new Row(6) testData.setField(0, BigDecimal("123456789.123456789123456789").bigDecimal) testData.setField(1, BigDecimal("123456789123456789123456789").bigDecimal) @@ -298,7 +305,7 @@ class DecimalTypeTest extends ExpressionTestBase { testData } - def typeInfo = { + def typeInfo: TypeInformation[Any] = { new RowTypeInfo( Types.DECIMAL, Types.DECIMAL, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/AvgFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/AvgFunctionTest.scala index 0671b40430684..d413c6c0c46f1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/AvgFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggfunctions/AvgFunctionTest.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.runtime.aggfunctions -import java.math.BigDecimal +import java.math.{BigDecimal, MathContext} import org.apache.flink.table.functions.AggregateFunction import org.apache.flink.table.functions.aggfunctions._ @@ -178,17 +178,23 @@ class DecimalAvgAggFunctionTest extends AggFunctionTestBase[BigDecimal, DecimalA null, null, null + ), + Seq( + new BigDecimal("0.3"), + new BigDecimal("0.3"), + new BigDecimal("0.4") ) ) override def expectedResults: Seq[BigDecimal] = Seq( BigDecimal.ZERO, BigDecimal.ONE, - null + null, + BigDecimal.ONE.divide(new BigDecimal("3"), MathContext.DECIMAL128) ) override def aggregator: AggregateFunction[BigDecimal, DecimalAvgAccumulator] = - new DecimalAvgAggFunction() + new DecimalAvgAggFunction(MathContext.DECIMAL128) override def retractFunc = aggregator.getClass.getMethod("retract", accType, classOf[Any]) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala index 1b89229fb9e46..aa37d1bd43dcd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime.batch.table +import java.math.MathContext import java.sql.{Date, Time, Timestamp} import java.util @@ -41,6 +42,7 @@ import org.junit.runners.Parameterized import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.math.BigDecimal.RoundingMode @RunWith(classOf[Parameterized]) class CalcITCase( @@ -330,6 +332,7 @@ class CalcITCase( def testAdvancedDataTypes(): Unit = { val env = ExecutionEnvironment.getExecutionEnvironment val tEnv = TableEnvironment.getTableEnvironment(env, config) + tEnv.getConfig.setDecimalContext(new MathContext(30)) val t = env .fromElements(( @@ -341,10 +344,11 @@ class CalcITCase( .toTable(tEnv, 'a, 'b, 'c, 'd, 'e) .select('a, 'b, 'c, 'd, 'e, BigDecimal("11.2"), BigDecimal("11.2").bigDecimal, Date.valueOf("1984-07-12"), Time.valueOf("14:34:24"), - Timestamp.valueOf("1984-07-12 14:34:24")) + Timestamp.valueOf("1984-07-12 14:34:24"), + BigDecimal("1").toExpr / BigDecimal("3")) val expected = "78.454654654654654,4E+9999,1984-07-12,14:34:24,1984-07-12 14:34:24.0," + - "11.2,11.2,1984-07-12,14:34:24,1984-07-12 14:34:24.0" + "11.2,11.2,1984-07-12,14:34:24,1984-07-12 14:34:24.0,0.333333333333333333333333333333" val results = t.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } From 491f59ef7592509a4c182dc28e7130bf6a8dbb7b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 2 Mar 2018 20:19:38 +0100 Subject: [PATCH 052/268] [FLINK-8842] Change the Rest default port to 8081 This closes #5626. --- .../flink/configuration/RestOptions.java | 2 +- .../entrypoint/ClusterConfiguration.java | 9 +++++++- .../runtime/entrypoint/ClusterEntrypoint.java | 22 +++++++++++++++++-- 3 files changed, 29 insertions(+), 4 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java index a66b27c7bb515..888be08239888 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java @@ -41,7 +41,7 @@ public class RestOptions { */ public static final ConfigOption REST_PORT = key("rest.port") - .defaultValue(9065) + .defaultValue(8081) .withDescription("The port that the server listens on / the client connects to."); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterConfiguration.java index dbec0b6a9eff9..7f8b5096d136c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterConfiguration.java @@ -27,11 +27,18 @@ public class ClusterConfiguration { private final String configDir; - public ClusterConfiguration(String configDir) { + private final int restPort; + + public ClusterConfiguration(String configDir, int restPort) { this.configDir = Preconditions.checkNotNull(configDir); + this.restPort = restPort; } public String getConfigDir() { return configDir; } + + public int getRestPort() { + return restPort; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index ba286703412c7..19781f8ee6471 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.WebOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.akka.AkkaUtils; @@ -594,11 +595,28 @@ protected static ClusterConfiguration parseArguments(String[] args) { final String configDir = parameterTool.get("configDir", ""); - return new ClusterConfiguration(configDir); + final int restPort; + + final String portKey = "webui-port"; + if (parameterTool.has(portKey)) { + restPort = Integer.valueOf(parameterTool.get(portKey)); + } else { + restPort = -1; + } + + return new ClusterConfiguration(configDir, restPort); } protected static Configuration loadConfiguration(ClusterConfiguration clusterConfiguration) { - return GlobalConfiguration.loadConfiguration(clusterConfiguration.getConfigDir()); + final Configuration configuration = GlobalConfiguration.loadConfiguration(clusterConfiguration.getConfigDir()); + + final int restPort = clusterConfiguration.getRestPort(); + + if (restPort >= 0) { + configuration.setInteger(RestOptions.REST_PORT, restPort); + } + + return configuration; } /** From 69ff5a7446a2ba8d266ed6286ad4553d95d35da0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 2 Mar 2018 20:31:36 +0100 Subject: [PATCH 053/268] [hotfix] Enable standalone HA mode by choosing HA port range --- .../flink/runtime/entrypoint/ClusterEntrypoint.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 19781f8ee6471..07b3b683a3f28 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.WebOptions; @@ -59,6 +60,7 @@ import org.apache.flink.runtime.security.SecurityConfiguration; import org.apache.flink.runtime.security.SecurityContext; import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.runtime.webmonitor.WebMonitorEndpoint; import org.apache.flink.runtime.webmonitor.retriever.LeaderGatewayRetriever; import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; @@ -354,7 +356,11 @@ protected void startClusterComponents( * @return Port range for the common {@link RpcService} */ protected String getRPCPortRange(Configuration configuration) { - return String.valueOf(configuration.getInteger(JobManagerOptions.PORT)); + if (ZooKeeperUtils.isZooKeeperRecoveryMode(configuration)) { + return configuration.getString(HighAvailabilityOptions.HA_JOB_MANAGER_PORT_RANGE); + } else { + return String.valueOf(configuration.getInteger(JobManagerOptions.PORT)); + } } protected RpcService createRpcService( From 2532b11ced3a7f541a4e1762031bfe136efb4d5b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 2 Mar 2018 15:27:13 +0100 Subject: [PATCH 054/268] [FLINK-8840] [yarn] Pull YarnClient and YarnConfiguration instantiation out of AbstractYarnClusterClient For better testability, this commit moves the YarnClient and YarnConfiguration out of the AbstractYarnClusterDescriptor. --- .../yarn/TestingYarnClusterDescriptor.java | 12 ++- .../yarn/YARNHighAvailabilityITCase.java | 7 +- .../org/apache/flink/yarn/YARNITCase.java | 6 +- .../flink/yarn/YARNSessionFIFOITCase.java | 5 +- .../org/apache/flink/yarn/YarnTestBase.java | 10 ++- .../yarn/AbstractYarnClusterDescriptor.java | 16 ++-- .../yarn/Flip6YarnClusterDescriptor.java | 12 ++- .../flink/yarn/YarnClusterDescriptor.java | 12 ++- .../flink/yarn/cli/FlinkYarnSessionCli.java | 30 ++++++- .../flink/yarn/AbstractYarnClusterTest.java | 20 ++++- .../flink/yarn/YarnClusterDescriptorTest.java | 79 ++++++++++++++++--- 11 files changed, 175 insertions(+), 34 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java index ec41d8e12b9bb..4d2aaa02efe34 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java @@ -24,6 +24,7 @@ import org.apache.flink.util.Preconditions; import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.io.File; import java.io.FilenameFilter; @@ -37,11 +38,18 @@ */ public class TestingYarnClusterDescriptor extends YarnClusterDescriptor { - public TestingYarnClusterDescriptor(Configuration configuration, String configurationDirectory) { + public TestingYarnClusterDescriptor( + Configuration configuration, + YarnConfiguration yarnConfiguration, + String configurationDirectory, + YarnClient yarnClient, + boolean sharedYarnClient) { super( configuration, + yarnConfiguration, configurationDirectory, - YarnClient.createYarnClient()); + yarnClient, + sharedYarnClient); List filesToShip = new ArrayList<>(); File testingJar = YarnTestBase.findFile("..", new TestJarFinder("flink-yarn-tests")); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index 05be03a175561..f9c03f937849f 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -110,7 +110,12 @@ public void testMultipleAMKill() throws Exception { final int numberKillingAttempts = numberApplicationAttempts - 1; String confDirPath = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); final Configuration configuration = GlobalConfiguration.loadConfiguration(); - TestingYarnClusterDescriptor flinkYarnClient = new TestingYarnClusterDescriptor(configuration, confDirPath); + TestingYarnClusterDescriptor flinkYarnClient = new TestingYarnClusterDescriptor( + configuration, + getYarnConfiguration(), + confDirPath, + getYarnClient(), + true); Assert.assertNotNull("unable to get yarn client", flinkYarnClient); flinkYarnClient.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java index 037e086ae8583..ef6706ad5fe36 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java @@ -55,12 +55,14 @@ public static void setup() { public void testPerJobMode() throws Exception { Configuration configuration = new Configuration(); configuration.setString(AkkaOptions.ASK_TIMEOUT, "30 s"); - final YarnClient yarnClient = YarnClient.createYarnClient(); + final YarnClient yarnClient = getYarnClient(); try (final Flip6YarnClusterDescriptor flip6YarnClusterDescriptor = new Flip6YarnClusterDescriptor( configuration, + getYarnConfiguration(), System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR), - yarnClient)) { + yarnClient, + true)) { flip6YarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); flip6YarnClusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index e54518793526d..b3dcaca145965 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -231,12 +231,13 @@ public void testJavaAPI() throws Exception { String confDirPath = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); Configuration configuration = GlobalConfiguration.loadConfiguration(); - final YarnClient yarnClient = YarnClient.createYarnClient(); try (final AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( configuration, + getYarnConfiguration(), confDirPath, - yarnClient)) { + getYarnClient(), + true)) { Assert.assertNotNull("unable to get yarn client", clusterDescriptor); clusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); clusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 7bca32192489b..b74a1557bb22b 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -128,7 +128,7 @@ public abstract class YarnTestBase extends TestLogger { */ protected static File flinkUberjar; - protected static final Configuration YARN_CONFIGURATION; + protected static final YarnConfiguration YARN_CONFIGURATION; /** * lib/ folder of the flink distribution. @@ -213,6 +213,14 @@ public void checkClusterEmpty() throws IOException, YarnException { flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(flinkConfiguration.getString(CoreOptions.MODE)); } + protected YarnClient getYarnClient() { + return yarnClient; + } + + protected static YarnConfiguration getYarnConfiguration() { + return YARN_CONFIGURATION; + } + /** * Locate a file or directory. */ diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index 6b930163896c0..bdb59b11b5fd7 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -118,6 +118,9 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor private final YarnClient yarnClient; + /** True if the descriptor must not shut down the YarnClient. */ + private final boolean sharedYarnClient; + private String yarnQueue; private String configurationDirectory; @@ -145,10 +148,12 @@ public abstract class AbstractYarnClusterDescriptor implements ClusterDescriptor public AbstractYarnClusterDescriptor( Configuration flinkConfiguration, + YarnConfiguration yarnConfiguration, String configurationDirectory, - YarnClient yarnClient) { + YarnClient yarnClient, + boolean sharedYarnClient) { - yarnConfiguration = new YarnConfiguration(); + this.yarnConfiguration = Preconditions.checkNotNull(yarnConfiguration); // for unit tests only if (System.getenv("IN_TESTS") != null) { @@ -160,8 +165,7 @@ public AbstractYarnClusterDescriptor( } this.yarnClient = Preconditions.checkNotNull(yarnClient); - yarnClient.init(yarnConfiguration); - yarnClient.start(); + this.sharedYarnClient = sharedYarnClient; this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); userJarInclusion = getUserJarInclusionMode(flinkConfiguration); @@ -328,7 +332,9 @@ public void setZookeeperNamespace(String zookeeperNamespace) { @Override public void close() { - yarnClient.stop(); + if (!sharedYarnClient) { + yarnClient.stop(); + } } // ------------------------------------------------------------- diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java index 461dd555a45e7..9860363c00e32 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java @@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; /** * Implementation of {@link org.apache.flink.yarn.AbstractYarnClusterDescriptor} which is used to start the @@ -39,9 +40,16 @@ public class Flip6YarnClusterDescriptor extends AbstractYarnClusterDescriptor { public Flip6YarnClusterDescriptor( Configuration flinkConfiguration, + YarnConfiguration yarnConfiguration, String configurationDirectory, - YarnClient yarnCLient) { - super(flinkConfiguration, configurationDirectory, yarnCLient); + YarnClient yarnClient, + boolean sharedYarnClient) { + super( + flinkConfiguration, + yarnConfiguration, + configurationDirectory, + yarnClient, + sharedYarnClient); } @Override diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index a5254a0e3a376..8625cee8240c6 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -26,6 +26,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; /** * Default implementation of {@link AbstractYarnClusterDescriptor} which starts an {@link YarnApplicationMasterRunner}. @@ -34,9 +35,16 @@ public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor { public YarnClusterDescriptor( Configuration flinkConfiguration, + YarnConfiguration yarnConfiguration, String configurationDirectory, - YarnClient yarnClient) { - super(flinkConfiguration, configurationDirectory, yarnClient); + YarnClient yarnClient, + boolean sharedYarnClient) { + super( + flinkConfiguration, + yarnConfiguration, + configurationDirectory, + yarnClient, + sharedYarnClient); } @Override diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index e4e3dbd5566cb..7773600dabd71 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.util.ConverterUtils; import org.slf4j.Logger; @@ -159,6 +160,8 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine Date: Thu, 1 Mar 2018 20:09:55 +0100 Subject: [PATCH 055/268] [hotfix] [flip6] Harden JobMaster#triggerSavepoint Check first whether the CheckpointCoordinator has been set before triggering a savepoint. If it has not been set, then return a failure message. --- .../flink/runtime/jobmaster/JobMaster.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index bfe8aaaa2424a..cc4cdbae15aaa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -971,14 +971,20 @@ public CompletableFuture requestJob(Time timeout) { @Override public CompletableFuture triggerSavepoint( - @Nullable final String targetDirectory, - final Time timeout) { - try { - return executionGraph.getCheckpointCoordinator() + @Nullable final String targetDirectory, + final Time timeout) { + final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); + + if (checkpointCoordinator != null) { + return checkpointCoordinator .triggerSavepoint(System.currentTimeMillis(), targetDirectory) .thenApply(CompletedCheckpoint::getExternalPointer); - } catch (Exception e) { - return FutureUtils.completedExceptionally(e); + } else { + return FutureUtils.completedExceptionally( + new FlinkException( + String.format( + "Cannot trigger a savepoint because the job %s is not a streaming job.", + jobGraph.getJobID()))); } } From b5c0fa808ff471f3040558bee5f456196ba61a02 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 1 Mar 2018 23:35:25 +0100 Subject: [PATCH 056/268] [FLINK-8826] [flip6] Start Yarn TaskExecutor with proper slots and memory Read the default TaskManager memory and number of slots from the configuration when the YarnResourceManager is started. This closes #5625. --- flink-end-to-end-tests/test-scripts/common.sh | 3 + flink-yarn-tests/pom.xml | 8 + .../flink/yarn/YarnConfigurationITCase.java | 194 ++++++++++++++++++ .../org/apache/flink/yarn/YarnTestBase.java | 31 ++- .../yarn/AbstractYarnClusterDescriptor.java | 8 +- .../flink/yarn/YarnResourceManager.java | 27 +-- .../flink/yarn/FlinkYarnSessionCliTest.java | 64 ++++++ 7 files changed, 312 insertions(+), 23 deletions(-) create mode 100644 flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index 7492f365afde1..e6d21a26b0907 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -75,6 +75,7 @@ function stop_cluster { | grep -v "RejectedExecutionException" \ | grep -v "An exception was thrown by an exception handler" \ | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/yarn/exceptions/YarnException" \ + | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" \ | grep -iq "error"; then echo "Found error in log files:" cat $FLINK_DIR/log/* @@ -92,7 +93,9 @@ function stop_cluster { | grep -v "RejectedExecutionException" \ | grep -v "An exception was thrown by an exception handler" \ | grep -v "Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.yarn.exceptions.YarnException" \ + | grep -v "Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.conf.Configuration" \ | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/yarn/exceptions/YarnException" \ + | grep -v "java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration" \ | grep -iq "exception"; then echo "Found exception in log files:" cat $FLINK_DIR/log/* diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index 6af7126c64bb0..cd76cb957ae15 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -378,6 +378,14 @@ under the License. true StreamingWordCount.jar + + org.apache.flink + flink-examples-streaming_${scala.binary.version} + jar + WindowJoin + true + WindowJoin.jar + ${project.build.directory}/programs false diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java new file mode 100644 index 0000000000000..2a1b099399ac0 --- /dev/null +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java @@ -0,0 +1,194 @@ +/* + * 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.yarn; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.rest.RestClient; +import org.apache.flink.runtime.rest.RestClientConfiguration; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; +import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; +import org.apache.flink.runtime.testingUtils.TestingUtils; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ContainerReport; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.net.URI; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +/** + * Test cases which ensure that the Yarn containers are started with the correct + * settings. + */ +public class YarnConfigurationITCase extends YarnTestBase { + + private static final Time TIMEOUT = Time.seconds(10L); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + /** + * Tests that the Flink components are started with the correct + * memory settings. + */ + @Test(timeout = 60000) + public void testFlinkContainerMemory() throws Exception { + final YarnClient yarnClient = getYarnClient(); + final Configuration configuration = new Configuration(flinkConfiguration); + + final int masterMemory = 64; + final int taskManagerMemory = 128; + final int slotsPerTaskManager = 3; + + // disable heap cutoff min + configuration.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 0); + configuration.setLong(TaskManagerOptions.NETWORK_BUFFERS_MEMORY_MIN, (1L << 20)); + configuration.setLong(TaskManagerOptions.NETWORK_BUFFERS_MEMORY_MAX, (4L << 20)); + + final YarnConfiguration yarnConfiguration = getYarnConfiguration(); + final Flip6YarnClusterDescriptor clusterDescriptor = new Flip6YarnClusterDescriptor( + configuration, + yarnConfiguration, + CliFrontend.getConfigurationDirectoryFromEnv(), + yarnClient, + true); + + clusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); + clusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); + + final File streamingWordCountFile = new File("target/programs/WindowJoin.jar"); + + assertThat(streamingWordCountFile.exists(), is(true)); + + final PackagedProgram packagedProgram = new PackagedProgram(streamingWordCountFile); + final JobGraph jobGraph = PackagedProgramUtils.createJobGraph(packagedProgram, configuration, 1); + + try { + final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(masterMemory) + .setTaskManagerMemoryMB(taskManagerMemory) + .setSlotsPerTaskManager(slotsPerTaskManager) + .createClusterSpecification(); + + final ClusterClient clusterClient = clusterDescriptor.deployJobCluster(clusterSpecification, jobGraph, true); + + final ApplicationId clusterId = clusterClient.getClusterId(); + + final RestClient restClient = new RestClient(RestClientConfiguration.fromConfiguration(configuration), TestingUtils.defaultExecutor()); + + try { + final ApplicationReport applicationReport = yarnClient.getApplicationReport(clusterId); + + final ApplicationAttemptId currentApplicationAttemptId = applicationReport.getCurrentApplicationAttemptId(); + + // wait until we have second container allocated + List containers = yarnClient.getContainers(currentApplicationAttemptId); + + while (containers.size() < 2) { + // this is nasty but Yarn does not offer a better way to wait + Thread.sleep(50L); + containers = yarnClient.getContainers(currentApplicationAttemptId); + } + + for (ContainerReport container : containers) { + if (container.getContainerId().getId() == 1) { + // this should be the application master + assertThat(container.getAllocatedResource().getMemory(), is(masterMemory)); + } else { + assertThat(container.getAllocatedResource().getMemory(), is(taskManagerMemory)); + } + } + + final URI webURI = new URI(clusterClient.getWebInterfaceURL()); + + CompletableFuture taskManagersInfoCompletableFuture; + Collection taskManagerInfos; + + while (true) { + taskManagersInfoCompletableFuture = restClient.sendRequest( + webURI.getHost(), + webURI.getPort(), + TaskManagersHeaders.getInstance(), + EmptyMessageParameters.getInstance(), + EmptyRequestBody.getInstance()); + + final TaskManagersInfo taskManagersInfo = taskManagersInfoCompletableFuture.get(); + + taskManagerInfos = taskManagersInfo.getTaskManagerInfos(); + + if (taskManagerInfos.isEmpty()) { + Thread.sleep(100L); + } else { + break; + } + } + + // there should be at least one TaskManagerInfo + final TaskManagerInfo taskManagerInfo = taskManagerInfos.iterator().next(); + + assertThat(taskManagerInfo.getNumberSlots(), is(slotsPerTaskManager)); + + final ContaineredTaskManagerParameters containeredTaskManagerParameters = ContaineredTaskManagerParameters.create( + configuration, + taskManagerMemory, + slotsPerTaskManager); + + final long expectedHeadSize = containeredTaskManagerParameters.taskManagerHeapSizeMB() << 20L; + + assertThat((double) taskManagerInfo.getHardwareDescription().getSizeOfJvmHeap() / (double) expectedHeadSize, is(closeTo(1.0, 0.1))); + } finally { + restClient.shutdown(TIMEOUT); + clusterClient.shutdown(); + } + + clusterDescriptor.terminateCluster(clusterId); + + } finally { + clusterDescriptor.close(); + } + } +} diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index b74a1557bb22b..3ec805e5058c5 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -55,6 +55,8 @@ import org.slf4j.Marker; import org.slf4j.MarkerFactory; +import javax.annotation.Nullable; + import java.io.BufferedWriter; import java.io.ByteArrayOutputStream; import java.io.File; @@ -73,6 +75,7 @@ import java.util.List; import java.util.Map; import java.util.Scanner; +import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.regex.Pattern; @@ -140,9 +143,15 @@ public abstract class YarnTestBase extends TestLogger { */ protected static File tempConfPathForSecureRun = null; + private YarnClient yarnClient = null; + + protected org.apache.flink.configuration.Configuration flinkConfiguration; + + protected boolean flip6; + static { YARN_CONFIGURATION = new YarnConfiguration(); - YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 512); + YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 32); YARN_CONFIGURATION.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 4096); // 4096 is the available memory anyways YARN_CONFIGURATION.setBoolean(YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS, true); YARN_CONFIGURATION.setBoolean(YarnConfiguration.RM_SCHEDULER_INCLUDE_PORT_IN_NODE_NAME, true); @@ -186,15 +195,11 @@ public void sleep() { } } - private YarnClient yarnClient = null; - protected org.apache.flink.configuration.Configuration flinkConfiguration; - protected boolean flip6; - @Before public void checkClusterEmpty() throws IOException, YarnException { if (yarnClient == null) { yarnClient = YarnClient.createYarnClient(); - yarnClient.init(YARN_CONFIGURATION); + yarnClient.init(getYarnConfiguration()); yarnClient.start(); } @@ -213,6 +218,7 @@ public void checkClusterEmpty() throws IOException, YarnException { flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(flinkConfiguration.getString(CoreOptions.MODE)); } + @Nullable protected YarnClient getYarnClient() { return yarnClient; } @@ -409,15 +415,15 @@ public static int getRunningContainers() { return count; } - public static void startYARNSecureMode(Configuration conf, String principal, String keytab) { + public static void startYARNSecureMode(YarnConfiguration conf, String principal, String keytab) { start(conf, principal, keytab); } - public static void startYARNWithConfig(Configuration conf) { + public static void startYARNWithConfig(YarnConfiguration conf) { start(conf, null, null); } - private static void start(Configuration conf, String principal, String keytab) { + private static void start(YarnConfiguration conf, String principal, String keytab) { // set the home directory to a temp directory. Flink on YARN is using the home dir to distribute the file File homeDir = null; try { @@ -444,7 +450,12 @@ private static void start(Configuration conf, String principal, String keytab) { try { LOG.info("Starting up MiniYARNCluster"); if (yarnCluster == null) { - yarnCluster = new MiniYARNCluster(conf.get(YarnTestBase.TEST_CLUSTER_NAME_KEY), NUM_NODEMANAGERS, 1, 1); + final String testName = conf.get(YarnTestBase.TEST_CLUSTER_NAME_KEY); + yarnCluster = new MiniYARNCluster( + testName == null ? "YarnTest_" + UUID.randomUUID() : testName, + NUM_NODEMANAGERS, + 1, + 1); yarnCluster.init(conf); yarnCluster.start(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index bdb59b11b5fd7..bdb471a142f96 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -33,6 +33,7 @@ import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; @@ -800,10 +801,15 @@ public ApplicationReport startAppMaster( homeDir, ""); + // set the right configuration values for the TaskManager configuration.setInteger( - ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, + TaskManagerOptions.NUM_TASK_SLOTS, clusterSpecification.getSlotsPerTaskManager()); + configuration.setInteger( + TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, + clusterSpecification.getTaskManagerMemoryMB()); + // Upload the flink configuration // write out configuration file File tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index f3ec04bf1e6e0..46ef81bed1707 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -20,6 +20,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -75,18 +76,9 @@ public class YarnResourceManager extends ResourceManager impleme /** YARN container map. Package private for unit test purposes. */ final ConcurrentMap workerNodeMap; - /** The default registration timeout for task executor in seconds. */ - private static final int DEFAULT_TASK_MANAGER_REGISTRATION_DURATION = 300; - /** The heartbeat interval while the resource master is waiting for containers. */ private static final int FAST_YARN_HEARTBEAT_INTERVAL_MS = 500; - /** The default heartbeat interval during regular operation. */ - private static final int DEFAULT_YARN_HEARTBEAT_INTERVAL_MS = 5000; - - /** The default memory of task executor to allocate (in MB). */ - private static final int DEFAULT_TSK_EXECUTOR_MEMORY_SIZE = 1024; - /** Environment variable name of the final container id used by the YarnResourceManager. * Container ID generation may vary across Hadoop versions. */ static final String ENV_FLINK_CONTAINER_ID = "_FLINK_CONTAINER_ID"; @@ -105,6 +97,12 @@ public class YarnResourceManager extends ResourceManager impleme @Nullable private final String webInterfaceUrl; + private final int defaultTaskManagerMemoryMB; + + private final int defaultNumSlots; + + private final int defaultCpus; + /** Client to communicate with the Resource Manager (YARN's master). */ private AMRMClientAsync resourceManagerClient; @@ -163,6 +161,9 @@ public YarnResourceManager( numPendingContainerRequests = 0; this.webInterfaceUrl = webInterfaceUrl; + this.defaultTaskManagerMemoryMB = flinkConfig.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY); + this.defaultNumSlots = flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); + this.defaultCpus = flinkConfig.getInteger(YarnConfigOptions.VCORES, defaultNumSlots); } protected AMRMClientAsync createAndStartResourceManagerClient( @@ -285,8 +286,8 @@ public void startNewWorker(ResourceProfile resourceProfile) { // Priority for worker containers - priorities are intra-application //TODO: set priority according to the resource allocated Priority priority = Priority.newInstance(generatePriority(resourceProfile)); - int mem = resourceProfile.getMemoryInMB() < 0 ? DEFAULT_TSK_EXECUTOR_MEMORY_SIZE : (int) resourceProfile.getMemoryInMB(); - int vcore = resourceProfile.getCpuCores() < 1 ? 1 : (int) resourceProfile.getCpuCores(); + int mem = resourceProfile.getMemoryInMB() < 0 ? defaultTaskManagerMemoryMB : (int) resourceProfile.getMemoryInMB(); + int vcore = resourceProfile.getCpuCores() < 1 ? defaultCpus : (int) resourceProfile.getCpuCores(); Resource capability = Resource.newInstance(mem, vcore); requestYarnContainer(capability, priority); } @@ -445,8 +446,10 @@ private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource // init the ContainerLaunchContext final String currDir = env.get(ApplicationConstants.Environment.PWD.key()); + final int numSlots = flinkConfig.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); + final ContaineredTaskManagerParameters taskManagerParameters = - ContaineredTaskManagerParameters.create(flinkConfig, resource.getMemory(), 1); + ContaineredTaskManagerParameters.create(flinkConfig, resource.getMemory(), numSlots); log.info("TaskExecutor {} will be started with container size {} MB, JVM heap size {} MB, " + "JVM direct memory limit {} MB", diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index 0a02474243a16..20ce314399f5b 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -21,6 +21,8 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; @@ -42,7 +44,9 @@ import java.nio.file.StandardOpenOption; import java.util.Map; +import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; /** @@ -249,6 +253,66 @@ public void testYarnIDOverridesPropertiesFile() throws Exception { assertEquals(TEST_YARN_APPLICATION_ID_2, clusterId); } + /** + * Tests that the command line arguments override the configuration settings + * when the {@link ClusterSpecification} is created. + */ + @Test + public void testCommandLineClusterSpecification() throws Exception { + final Configuration configuration = new Configuration(); + configuration.setInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, 1337); + configuration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, 7331); + configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); + + final int jobManagerMemory = 42; + final int taskManagerMemory = 41; + final int slotsPerTaskManager = 30; + final String[] args = {"-yjm", String.valueOf(jobManagerMemory), "-ytm", String.valueOf(taskManagerMemory), "-ys", String.valueOf(slotsPerTaskManager)}; + final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( + configuration, + tmp.getRoot().getAbsolutePath(), + "y", + "yarn"); + + CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + + assertThat(clusterSpecification.getMasterMemoryMB(), is(jobManagerMemory)); + assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(taskManagerMemory)); + assertThat(clusterSpecification.getSlotsPerTaskManager(), is(slotsPerTaskManager)); + } + + /** + * Tests that the configuration settings are used to create the + * {@link ClusterSpecification}. + */ + @Test + public void testConfigurationClusterSpecification() throws Exception { + final Configuration configuration = new Configuration(); + final int jobManagerMemory = 1337; + configuration.setInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, jobManagerMemory); + final int taskManagerMemory = 7331; + configuration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, taskManagerMemory); + final int slotsPerTaskManager = 42; + configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, slotsPerTaskManager); + + final String[] args = {}; + final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( + configuration, + tmp.getRoot().getAbsolutePath(), + "y", + "yarn"); + + CommandLine commandLine = flinkYarnSessionCli.parseCommandLineOptions(args, false); + + final ClusterSpecification clusterSpecification = flinkYarnSessionCli.getClusterSpecification(commandLine); + + assertThat(clusterSpecification.getMasterMemoryMB(), is(jobManagerMemory)); + assertThat(clusterSpecification.getTaskManagerMemoryMB(), is(taskManagerMemory)); + assertThat(clusterSpecification.getSlotsPerTaskManager(), is(slotsPerTaskManager)); + } + /////////// // Utils // /////////// From 75b092db3b897aaaf3e13ccaaf8bfc51b8af00ca Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 2 Mar 2018 12:18:05 +0100 Subject: [PATCH 057/268] [hotfix] Set default number of TaskManagers in FlinkYarnSessionCli for Flip6 --- .../apache/flink/yarn/cli/FlinkYarnSessionCli.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 7773600dabd71..2cdc19d3c9360 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -378,7 +378,14 @@ private ClusterSpecification createClusterSpecification(Configuration configurat throw new IllegalArgumentException("Missing required argument " + container.getOpt()); } - int numberTaskManagers = Integer.valueOf(cmd.getOptionValue(container.getOpt())); + // TODO: The number of task manager should be deprecated soon + final int numberTaskManagers; + + if (cmd.hasOption(container.getOpt())) { + numberTaskManagers = Integer.valueOf(cmd.getOptionValue(container.getOpt())); + } else { + numberTaskManagers = 1; + } // JobManager Memory final int jobManagerMemoryMB = configuration.getInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY); @@ -386,7 +393,7 @@ private ClusterSpecification createClusterSpecification(Configuration configurat // Task Managers memory final int taskManagerMemoryMB = configuration.getInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY); - int slotsPerTaskManager = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + int slotsPerTaskManager = configuration.getInteger(TaskManagerOptions.NUM_TASK_SLOTS); return new ClusterSpecification.ClusterSpecificationBuilder() .setMasterMemoryMB(jobManagerMemoryMB) From 1916e263b1c6c77c6c980fe0a3e7ee9df1c1616c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 2 Mar 2018 12:42:43 +0100 Subject: [PATCH 058/268] [hotfix] Print correct web monitor URL in FlinkYarnSessionCli --- .../flink/client/program/rest/RestClusterClient.java | 10 +++++++++- .../java/org/apache/flink/util/ExceptionUtils.java | 12 ++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 976f2a4db31ca..5a0936d16ca3d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -101,6 +101,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -525,7 +526,14 @@ public void waitForClusterToBeReady() { @Override public String getWebInterfaceURL() { - return getWebMonitorBaseUrl().toString(); + try { + return getWebMonitorBaseUrl().get().toString(); + } catch (InterruptedException | ExecutionException e) { + ExceptionUtils.checkInterrupted(e); + + log.warn("Could not retrieve the web interface URL for the cluster.", e); + return "Unknown address."; + } } @Override diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java index 42deb69cf3632..b9a21ae349528 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java @@ -398,6 +398,18 @@ public static void tryDeserializeAndThrow(Throwable throwable, ClassLoader class } } + /** + * Checks whether the given exception is a {@link InterruptedException} and sets + * the interrupted flag accordingly. + * + * @param e to check whether it is an {@link InterruptedException} + */ + public static void checkInterrupted(Throwable e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + } + // ------------------------------------------------------------------------ // Lambda exception utilities // ------------------------------------------------------------------------ From 11c30c8cc9e1628032879731a6e851953754b8db Mon Sep 17 00:00:00 2001 From: gyao Date: Fri, 2 Mar 2018 15:11:36 +0100 Subject: [PATCH 059/268] [FLINK-8459][flip6] Implement RestClusterClient.cancelWithSavepoint Introduce cancelJob flag to existing triggerSavepoint methods in Dispatcher and JobMaster. Stop checkpoint scheduler before taking savepoint to make sure that the savepoint created by this command is the last one. This closes #5622. --- .../client/program/MiniClusterClient.java | 2 +- .../program/rest/RestClusterClient.java | 15 +- .../flink/runtime/dispatcher/Dispatcher.java | 3 +- .../executiongraph/ExecutionGraph.java | 1 + .../flink/runtime/jobmaster/JobMaster.java | 45 +++- .../runtime/jobmaster/JobMasterGateway.java | 1 + .../runtime/minicluster/MiniCluster.java | 11 + .../job/savepoints/SavepointHandlers.java | 3 +- .../SavepointTriggerRequestBody.java | 12 +- .../runtime/webmonitor/RestfulGateway.java | 1 + .../utils/TestingJobMasterGateway.java | 4 +- ...ractAsynchronousOperationHandlersTest.java | 2 +- .../job/savepoints/SavepointHandlersTest.java | 2 +- .../SavepointTriggerRequestBodyTest.java | 25 +- .../webmonitor/TestingRestfulGateway.java | 2 +- .../JobMasterTriggerSavepointIT.java | 219 ++++++++++++++++++ 16 files changed, 323 insertions(+), 25 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index aca75e0c15087..dd99f0dabeb99 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -87,7 +87,7 @@ public void cancel(JobID jobId) throws Exception { @Override public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) throws Exception { - throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + return miniCluster.triggerSavepoint(jobId, savepointDirectory, true).get(); } @Override diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 5a0936d16ca3d..560a10e955072 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -354,24 +354,29 @@ public void cancel(JobID jobID) throws Exception { @Override public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) throws Exception { - throw new UnsupportedOperationException("Not implemented yet."); + return triggerSavepoint(jobId, savepointDirectory, true).get(); } @Override public CompletableFuture triggerSavepoint( final JobID jobId, final @Nullable String savepointDirectory) throws FlinkException { + return triggerSavepoint(jobId, savepointDirectory, false); + } + + private CompletableFuture triggerSavepoint( + final JobID jobId, + final @Nullable String savepointDirectory, + final boolean cancelJob) { final SavepointTriggerHeaders savepointTriggerHeaders = SavepointTriggerHeaders.getInstance(); final SavepointTriggerMessageParameters savepointTriggerMessageParameters = savepointTriggerHeaders.getUnresolvedMessageParameters(); savepointTriggerMessageParameters.jobID.resolve(jobId); - final CompletableFuture responseFuture; - - responseFuture = sendRequest( + final CompletableFuture responseFuture = sendRequest( savepointTriggerHeaders, savepointTriggerMessageParameters, - new SavepointTriggerRequestBody(savepointDirectory)); + new SavepointTriggerRequestBody(savepointDirectory, cancelJob)); return responseFuture.thenCompose(savepointTriggerResponseBody -> { final TriggerId savepointTriggerId = savepointTriggerResponseBody.getTriggerId(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 7a11cf0878418..9b2411c66b789 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -487,11 +487,12 @@ public CompletableFuture getBlobServerPort(Time timeout) { public CompletableFuture triggerSavepoint( final JobID jobId, final String targetDirectory, + final boolean cancelJob, final Time timeout) { if (jobManagerRunners.containsKey(jobId)) { return jobManagerRunners.get(jobId) .getJobManagerGateway() - .triggerSavepoint(targetDirectory, timeout); + .triggerSavepoint(targetDirectory, cancelJob, timeout); } else { return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 4e8b972bc0786..ee23884d3a6f7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -524,6 +524,7 @@ public void enableCheckpointing( } } + @Nullable public CheckpointCoordinator getCheckpointCoordinator() { return checkpointCoordinator; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index cc4cdbae15aaa..74f9b656c6f0e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -972,19 +972,43 @@ public CompletableFuture requestJob(Time timeout) { @Override public CompletableFuture triggerSavepoint( @Nullable final String targetDirectory, + final boolean cancelJob, final Time timeout) { + final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); + if (checkpointCoordinator == null) { + return FutureUtils.completedExceptionally(new IllegalStateException( + String.format("Job %s is not a streaming job.", jobGraph.getJobID()))); + } - if (checkpointCoordinator != null) { - return checkpointCoordinator - .triggerSavepoint(System.currentTimeMillis(), targetDirectory) - .thenApply(CompletedCheckpoint::getExternalPointer); - } else { - return FutureUtils.completedExceptionally( - new FlinkException( - String.format( - "Cannot trigger a savepoint because the job %s is not a streaming job.", - jobGraph.getJobID()))); + if (cancelJob) { + checkpointCoordinator.stopCheckpointScheduler(); + } + return checkpointCoordinator + .triggerSavepoint(System.currentTimeMillis(), targetDirectory) + .thenApply(CompletedCheckpoint::getExternalPointer) + .thenApplyAsync(path -> { + if (cancelJob) { + log.info("Savepoint stored in {}. Now cancelling {}.", path, jobGraph.getJobID()); + cancel(timeout); + } + return path; + }, getMainThreadExecutor()) + .exceptionally(throwable -> { + if (cancelJob) { + startCheckpointScheduler(checkpointCoordinator); + } + throw new CompletionException(throwable); + }); + } + + private void startCheckpointScheduler(final CheckpointCoordinator checkpointCoordinator) { + if (checkpointCoordinator.isPeriodicCheckpointingConfigured()) { + try { + checkpointCoordinator.startCheckpointScheduler(); + } catch (IllegalStateException ignored) { + // Concurrent shut down of the coordinator + } } } @@ -1321,6 +1345,7 @@ private CompletableFuture restoreExecutionGraphFromRescalingSave private CompletableFuture getJobModificationSavepoint(Time timeout) { return triggerSavepoint( null, + false, timeout) .handleAsync( (String savepointPath, Throwable throwable) -> { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 6173a26cd8292..1e1bdda45117a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -263,6 +263,7 @@ CompletableFuture registerTaskManager( */ CompletableFuture triggerSavepoint( @Nullable final String targetDirectory, + final boolean cancelJob, final Time timeout); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 6b5f9b50aecca..98c8ca22e9ba4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -483,6 +483,17 @@ public CompletableFuture cancelJob(JobID jobId) { } } + public CompletableFuture triggerSavepoint(JobID jobId, String targetDirectory, boolean cancelJob) { + try { + return getDispatcherGateway().triggerSavepoint(jobId, targetDirectory, cancelJob, rpcTimeout); + } catch (LeaderRetrievalException | InterruptedException e) { + return FutureUtils.completedExceptionally( + new FlinkException( + String.format("Could not trigger savepoint for job %s.", jobId), + e)); + } + } + // ------------------------------------------------------------------------ // running jobs // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java index cb3ff5bb06c46..17e263bec6e63 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlers.java @@ -127,8 +127,9 @@ protected CompletableFuture triggerOperation(HandlerRequest requestMultipleJobDetails( default CompletableFuture triggerSavepoint( JobID jobId, String targetDirectory, + boolean cancelJob, @RpcTimeout Time timeout) { throw new UnsupportedOperationException(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java index cac7e90bd09b3..0d57a56b2ceb3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java @@ -48,6 +48,8 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import javax.annotation.Nullable; + import java.net.InetSocketAddress; import java.util.Collection; import java.util.concurrent.CompletableFuture; @@ -153,7 +155,7 @@ public CompletableFuture requestJob(Time timeout) { } @Override - public CompletableFuture triggerSavepoint(String targetDirectory, Time timeout) { + public CompletableFuture triggerSavepoint(@Nullable final String targetDirectory, final boolean cancelJob, final Time timeout) { throw new UnsupportedOperationException(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/async/AbstractAsynchronousOperationHandlersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/async/AbstractAsynchronousOperationHandlersTest.java index 848e2539d7b07..7ad140e6d20bd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/async/AbstractAsynchronousOperationHandlersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/async/AbstractAsynchronousOperationHandlersTest.java @@ -305,7 +305,7 @@ protected TestingTriggerHandler(CompletableFuture localRestAddress, Gate @Override protected CompletableFuture triggerOperation(HandlerRequest request, RestfulGateway gateway) throws RestHandlerException { - return gateway.triggerSavepoint(new JobID(), null, timeout); + return gateway.triggerSavepoint(new JobID(), null, false, timeout); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlersTest.java index a8f4b3f275406..06944525a39dd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointHandlersTest.java @@ -200,7 +200,7 @@ private static HandlerRequest( - new SavepointTriggerRequestBody(targetDirectory), + new SavepointTriggerRequestBody(targetDirectory, false), new SavepointTriggerMessageParameters(), Collections.singletonMap(JobIDPathParameter.KEY, JOB_ID.toString()), Collections.emptyMap()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointTriggerRequestBodyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointTriggerRequestBodyTest.java index f7c3973627c89..f79f8a2682188 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointTriggerRequestBodyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointTriggerRequestBodyTest.java @@ -20,22 +20,43 @@ import org.apache.flink.runtime.rest.messages.RestRequestMarshallingTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; + import static org.junit.Assert.assertEquals; /** * Tests for {@link SavepointTriggerRequestBody}. */ +@RunWith(Parameterized.class) public class SavepointTriggerRequestBodyTest extends RestRequestMarshallingTestBase { + private final SavepointTriggerRequestBody savepointTriggerRequestBody; + + public SavepointTriggerRequestBodyTest(final SavepointTriggerRequestBody savepointTriggerRequestBody) { + this.savepointTriggerRequestBody = savepointTriggerRequestBody; + } + + @Parameterized.Parameters + public static Collection data() { + return Arrays.asList(new Object[][]{ + {new SavepointTriggerRequestBody("/tmp", true)}, + {new SavepointTriggerRequestBody("/tmp", false)} + }); + } + @Override protected Class getTestRequestClass() { return SavepointTriggerRequestBody.class; } @Override - protected SavepointTriggerRequestBody getTestRequestInstance() throws Exception { - return new SavepointTriggerRequestBody("/tmp"); + protected SavepointTriggerRequestBody getTestRequestInstance() { + return savepointTriggerRequestBody; } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java index 5eff5a680e649..b92ba5182bee4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java @@ -190,7 +190,7 @@ public CompletableFuture requestOperatorBackP } @Override - public CompletableFuture triggerSavepoint(JobID jobId, String targetDirectory, Time timeout) { + public CompletableFuture triggerSavepoint(JobID jobId, String targetDirectory, boolean cancelJob, Time timeout) { return triggerSavepointFunction.apply(jobId, targetDirectory); } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java new file mode 100644 index 0000000000000..f9edfa6369540 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java @@ -0,0 +1,219 @@ +/* + * 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.jobmaster; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.program.MiniClusterClient; +import org.apache.flink.runtime.checkpoint.CheckpointMetaData; +import org.apache.flink.runtime.checkpoint.CheckpointMetrics; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; +import org.apache.flink.runtime.checkpoint.CheckpointTriggerException; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; +import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; +import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.util.ExceptionUtils; + +import org.junit.Assume; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.isOneOf; + +/** + * Tests for {@link org.apache.flink.runtime.jobmaster.JobMaster#triggerSavepoint(String, boolean, Time)}. + * + * @see org.apache.flink.runtime.jobmaster.JobMaster + */ +@Category(Flip6.class) +public class JobMasterTriggerSavepointIT extends AbstractTestBase { + + private static CountDownLatch invokeLatch; + + private static volatile CountDownLatch triggerCheckpointLatch; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private Path savepointDirectory; + private MiniClusterClient clusterClient; + private JobGraph jobGraph; + + @Before + public void setUp() throws Exception { + invokeLatch = new CountDownLatch(1); + triggerCheckpointLatch = new CountDownLatch(1); + savepointDirectory = temporaryFolder.newFolder().toPath(); + + Assume.assumeTrue( + "ClusterClient is not an instance of MiniClusterClient", + miniClusterResource.getClusterClient() instanceof MiniClusterClient); + + clusterClient = (MiniClusterClient) miniClusterResource.getClusterClient(); + clusterClient.setDetached(true); + + jobGraph = new JobGraph(); + + final JobVertex vertex = new JobVertex("testVertex"); + vertex.setInvokableClass(NoOpBlockingInvokable.class); + jobGraph.addVertex(vertex); + + jobGraph.setSnapshotSettings(new JobCheckpointingSettings( + Collections.singletonList(vertex.getID()), + Collections.singletonList(vertex.getID()), + Collections.singletonList(vertex.getID()), + new CheckpointCoordinatorConfiguration( + 10, + 60_000, + 10, + 1, + CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, + true), + null + )); + + clusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader()); + invokeLatch.await(60, TimeUnit.SECONDS); + waitForJob(); + } + + @Test + public void testStopJobAfterSavepoint() throws Exception { + final String savepointLocation = cancelWithSavepoint(); + final JobStatus jobStatus = clusterClient.getJobStatus(jobGraph.getJobID()).get(60, TimeUnit.SECONDS); + + assertThat(jobStatus, isOneOf(JobStatus.CANCELED, JobStatus.CANCELLING)); + + final List savepoints = Files.list(savepointDirectory).map(Path::getFileName).collect(Collectors.toList()); + assertThat(savepoints, hasItem(Paths.get(savepointLocation).getFileName())); + } + + @Test + public void testDoNotCancelJobIfSavepointFails() throws Exception { + try { + Files.setPosixFilePermissions(savepointDirectory, Collections.emptySet()); + } catch (IOException e) { + Assume.assumeNoException(e); + } + + try { + cancelWithSavepoint(); + } catch (Exception e) { + assertThat(ExceptionUtils.findThrowable(e, CheckpointTriggerException.class).isPresent(), equalTo(true)); + } + + final JobStatus jobStatus = clusterClient.getJobStatus(jobGraph.getJobID()).get(60, TimeUnit.SECONDS); + assertThat(jobStatus, equalTo(JobStatus.RUNNING)); + + // assert that checkpoints are continued to be triggered + triggerCheckpointLatch = new CountDownLatch(1); + assertThat(triggerCheckpointLatch.await(60, TimeUnit.SECONDS), equalTo(true)); + } + + private void waitForJob() throws Exception { + for (int i = 0; i < 60; i++) { + try { + final JobStatus jobStatus = clusterClient.getJobStatus(jobGraph.getJobID()).get(60, TimeUnit.SECONDS); + assertThat(jobStatus.isGloballyTerminalState(), equalTo(false)); + if (jobStatus == JobStatus.RUNNING) { + return; + } + } catch (ExecutionException ignored) { + // JobManagerRunner is not yet registered in Dispatcher + } + Thread.sleep(1000); + } + throw new AssertionError("Job did not become running within timeout."); + } + + /** + * Invokable which calls {@link CountDownLatch#countDown()} on + * {@link JobMasterTriggerSavepointIT#invokeLatch}, and then blocks afterwards. + */ + public static class NoOpBlockingInvokable extends AbstractInvokable { + + public NoOpBlockingInvokable(final Environment environment) { + super(environment); + } + + @Override + public void invoke() { + invokeLatch.countDown(); + try { + Thread.sleep(Long.MAX_VALUE); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + @Override + public boolean triggerCheckpoint(final CheckpointMetaData checkpointMetaData, final CheckpointOptions checkpointOptions) throws Exception { + final TaskStateSnapshot checkpointStateHandles = new TaskStateSnapshot(); + checkpointStateHandles.putSubtaskStateByOperatorID( + OperatorID.fromJobVertexID(getEnvironment().getJobVertexId()), + new OperatorSubtaskState()); + + getEnvironment().acknowledgeCheckpoint( + checkpointMetaData.getCheckpointId(), + new CheckpointMetrics(), + checkpointStateHandles); + + triggerCheckpointLatch.countDown(); + + return true; + } + + @Override + public void notifyCheckpointComplete(final long checkpointId) throws Exception { + } + } + + private String cancelWithSavepoint() throws Exception { + return clusterClient.cancelWithSavepoint( + jobGraph.getJobID(), + savepointDirectory.toAbsolutePath().toString()); + } + +} From 7c3be91491f9a0d2ed9665dc0908ed59ab1587ef Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 26 Feb 2018 11:52:50 +0100 Subject: [PATCH 060/268] [FLINK-8758] Make non-blocking ClusterClient.submitJob() public --- .../java/org/apache/flink/client/program/ClusterClient.java | 2 +- .../apache/flink/client/program/StandaloneClusterClient.java | 2 +- .../org/apache/flink/client/program/rest/RestClusterClient.java | 2 +- .../src/main/java/org/apache/flink/yarn/YarnClusterClient.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index 1cf2bc2847c36..7817f8f85d079 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -987,7 +987,7 @@ public Configuration getFlinkConfiguration() { * @param jobGraph The JobGraph to be submitted * @return JobSubmissionResult */ - protected abstract JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) + public abstract JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException; /** diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java index ee8ad44ddc0e1..1c9c690710ee1 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java @@ -107,7 +107,7 @@ public boolean hasUserJarsInClassPath(List userJarFiles) { } @Override - protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) + public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { if (isDetached()) { return super.runDetached(jobGraph, classLoader); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 560a10e955072..18ff0992cb685 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -219,7 +219,7 @@ public void shutdown() { } @Override - protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { + public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { log.info("Submitting job {}.", jobGraph.getJobID()); final CompletableFuture jobSubmissionFuture = submitJob(jobGraph); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java index f2be264fe20d7..e0010c769ae0e 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java @@ -148,7 +148,7 @@ public boolean hasUserJarsInClassPath(List userJarFiles) { } @Override - protected JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { + public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { if (isDetached()) { if (newlyCreatedCluster) { stopAfterJob(jobGraph.getJobID()); From 824eb02627f1e550356a07fcb630c3e0cd6da183 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 27 Feb 2018 17:29:00 +0100 Subject: [PATCH 061/268] [FLINK-8818][yarn/s3][tests] harden YarnFileStageTest upload test for eventual consistent read-after-write In case the newly written object cannot be read (yet), we do 4 more retries to retrieve the value and wait 50ms each. While this does not solve all the cases it should make the (rare) case of the written object not being available for read even more unlikely. (cherry picked from commit c74d8ca) --- .../apache/flink/yarn/YarnFileStageTest.java | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java index 5cbe1be7eeafb..527782c257aa8 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTest.java @@ -41,6 +41,7 @@ import java.io.DataOutputStream; import java.io.File; +import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.util.ArrayList; import java.util.Collections; @@ -200,13 +201,23 @@ static void testCopyFromLocalRecursive( while (targetFilesIterator.hasNext()) { LocatedFileStatus targetFile = targetFilesIterator.next(); - try (FSDataInputStream in = targetFileSystem.open(targetFile.getPath())) { - String absolutePathString = targetFile.getPath().toString(); - String relativePath = absolutePathString.substring(workDirPrefixLength); - targetFiles.put(relativePath, in.readUTF()); - - assertEquals("extraneous data in file " + relativePath, -1, in.read()); - } + int retries = 5; + do { + try (FSDataInputStream in = targetFileSystem.open(targetFile.getPath())) { + String absolutePathString = targetFile.getPath().toString(); + String relativePath = absolutePathString.substring(workDirPrefixLength); + targetFiles.put(relativePath, in.readUTF()); + + assertEquals("extraneous data in file " + relativePath, -1, in.read()); + break; + } catch (FileNotFoundException e) { + // For S3, read-after-write may be eventually consistent, i.e. when trying + // to access the object before writing it; see + // https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel + // -> try again a bit later + Thread.sleep(50); + } + } while ((retries--) > 0); } assertThat(targetFiles, equalTo(srcFiles)); From 8f995e7a0e35b6e12f546e24c895fe008a32baec Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 1 Mar 2018 13:53:21 +0100 Subject: [PATCH 062/268] [FLINK-8769][flip6] do not print error causing exceptions without debugging In DispatcherRestEndpoint and TaskExecutor, there were two places where without errors (running a job inside an IDE) exceptions were logged. While for debugging they may be useful, for normal operation it is enough to print the messages themselves, especially since some more details were already logged before. (cherry picked from commit fc0001c) --- .../dispatcher/DispatcherRestEndpoint.java | 7 ++++++- .../flink/runtime/taskexecutor/TaskExecutor.java | 16 ++++++++++++++-- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java index b5205ab28d870..9df6deec49fca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java @@ -120,7 +120,12 @@ protected List> initiali // register extension handlers handlers.addAll(webSubmissionExtension.getHandlers()); } catch (FlinkException e) { - log.info("Failed to load web based job submission extension.", e); + if (log.isDebugEnabled()) { + log.debug("Failed to load web based job submission extension.", e); + } else { + log.info("Failed to load web based job submission extension. " + + "Probable reason: flink-runtime-web is not in the classpath."); + } } } else { log.info("Web-based job submission is not enabled."); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index cab686af59aa9..fc69984c8a0c8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -913,13 +913,25 @@ private void closeResourceManagerConnection(Exception cause) { if (resourceManagerConnection != null) { if (resourceManagerConnection.isConnected()) { - log.info("Close ResourceManager connection {}.", resourceManagerConnection.getResourceManagerId(), cause); + if (log.isDebugEnabled()) { + log.debug("Close ResourceManager connection {}.", + resourceManagerConnection.getResourceManagerId(), cause); + } else { + log.info("Close ResourceManager connection {}.", + resourceManagerConnection.getResourceManagerId()); + } resourceManagerHeartbeatManager.unmonitorTarget(resourceManagerConnection.getResourceManagerId()); ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway(); resourceManagerGateway.disconnectTaskManager(getResourceID(), cause); } else { - log.info("Terminating registration attempts towards ResourceManager {}.", resourceManagerConnection.getTargetAddress(), cause); + if (log.isDebugEnabled()) { + log.debug("Terminating registration attempts towards ResourceManager {}.", + resourceManagerConnection.getTargetAddress(), cause); + } else { + log.info("Terminating registration attempts towards ResourceManager {}.", + resourceManagerConnection.getTargetAddress()); + } } resourceManagerConnection.close(); From 05f3fa45d8f806dfff1af7b8c442263f024ef259 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Fri, 23 Feb 2018 04:29:16 +0100 Subject: [PATCH 063/268] [FLINK-8458][config][docs] Add config of credit-based network buffers (cherry picked from commit 3c8a673) --- .../generated/netty_configuration.html | 15 +++++++++++++++ .../flink/configuration/TaskManagerOptions.java | 17 +++++++++++++---- 2 files changed, 28 insertions(+), 4 deletions(-) diff --git a/docs/_includes/generated/netty_configuration.html b/docs/_includes/generated/netty_configuration.html index 47c48c0aa38d2..e97fda45d1055 100644 --- a/docs/_includes/generated/netty_configuration.html +++ b/docs/_includes/generated/netty_configuration.html @@ -42,5 +42,20 @@ "nio" The Netty transport type, either "nio" or "epoll" + +

    taskmanager.network.credit-based-flow-control.enabled
    + true + Boolean flag to enable/disable network credit-based flow control + + +
    taskmanager.network.memory.buffers-per-channel
    + 2 + Number of network buffers to use for each outgoing/incoming channel (subpartition/input channel). In credit-based flow control mode, this indicates how many credits are exclusive in each input channel. It should be configured at least 2 for good performance. 1 buffer is for receiving in-flight data in the subpartition and 1 buffer is for parallel serialization + + +
    taskmanager.network.memory.floating-buffers-per-gate
    + 8 + Number of extra network buffers to use for each outgoing/incoming gate (result partition/input gate). In credit-based flow control mode, this indicates how many floating credits are shared among all the input channels. The floating buffers are distributed based on backlog (real-time output buffers in the subpartition) feedback, and can help relieve back-pressure caused by unbalanced data distribution among the subpartitions. This value should be increased in case of higher round trip times between nodes and/or larger number of machines in the cluster + diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java index cc3284cdbde7c..4e08fdaa9910a 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java @@ -269,7 +269,10 @@ public class TaskManagerOptions { public static final ConfigOption NETWORK_BUFFERS_PER_CHANNEL = key("taskmanager.network.memory.buffers-per-channel") .defaultValue(2) - .withDescription("Number of network buffers to use for each outgoing/incoming channel (subpartition/input channel)."); + .withDescription("Number of network buffers to use for each outgoing/incoming channel (subpartition/input channel)." + + "In credit-based flow control mode, this indicates how many credits are exclusive in each input channel. It should be" + + " configured at least 2 for good performance. 1 buffer is for receiving in-flight data in the subpartition and 1 buffer is" + + " for parallel serialization."); /** * Number of extra network buffers to use for each outgoing/incoming gate (result partition/input gate). @@ -277,7 +280,12 @@ public class TaskManagerOptions { public static final ConfigOption NETWORK_EXTRA_BUFFERS_PER_GATE = key("taskmanager.network.memory.floating-buffers-per-gate") .defaultValue(8) - .withDescription("Number of extra network buffers to use for each outgoing/incoming gate (result partition/input gate)."); + .withDescription("Number of extra network buffers to use for each outgoing/incoming gate (result partition/input gate)." + + " In credit-based flow control mode, this indicates how many floating credits are shared among all the input channels." + + " The floating buffers are distributed based on backlog (real-time output buffers in the subpartition) feedback, and can" + + " help relieve back-pressure caused by unbalanced data distribution among the subpartitions. This value should be" + + " increased in case of higher round trip times between nodes and/or larger number of machines in the cluster."); + /** * Minimum backoff for partition requests of input channels. @@ -307,7 +315,7 @@ public class TaskManagerOptions { .withDescription("Boolean flag to enable/disable more detailed metrics about inbound/outbound network queue lengths."); /** - * Config parameter defining whether to enable credit-based flow control or not. + * Boolean flag to enable/disable network credit-based flow control. * * @deprecated Will be removed for Flink 1.6 when the old code will be dropped in favour of * credit-based flow control. @@ -315,7 +323,8 @@ public class TaskManagerOptions { @Deprecated public static final ConfigOption NETWORK_CREDIT_BASED_FLOW_CONTROL_ENABLED = key("taskmanager.network.credit-based-flow-control.enabled") - .defaultValue(true); + .defaultValue(true) + .withDescription("Boolean flag to enable/disable network credit-based flow control."); /** * Config parameter defining whether to spill data for channels with barrier or not in exactly-once From 6b06292fb04b47a770ed83fab21032f9af7740ed Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Mon, 5 Mar 2018 13:21:27 +0100 Subject: [PATCH 064/268] [FLINK-8859][checkpointing] RocksDB backend should pass WriteOption to Rocks.put() when restoring (cherry picked from commit 131daa2) --- .../contrib/streaming/state/RocksDBKeyedStateBackend.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 8f95b1812d844..5444dee443851 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -680,7 +680,7 @@ private void restoreKVStateData() throws IOException, RocksDBException { if (RocksDBFullSnapshotOperation.hasMetaDataFollowsFlag(key)) { //clear the signal bit in the key to make it ready for insertion again RocksDBFullSnapshotOperation.clearMetaDataFollowsFlag(key); - rocksDBKeyedStateBackend.db.put(handle, key, value); + rocksDBKeyedStateBackend.db.put(handle, rocksDBKeyedStateBackend.writeOptions, key, value); //TODO this could be aware of keyGroupPrefixBytes and write only one byte if possible kvStateId = RocksDBFullSnapshotOperation.END_OF_KEY_GROUP_MARK & compressedKgInputView.readShort(); @@ -690,7 +690,7 @@ private void restoreKVStateData() throws IOException, RocksDBException { handle = currentStateHandleKVStateColumnFamilies.get(kvStateId); } } else { - rocksDBKeyedStateBackend.db.put(handle, key, value); + rocksDBKeyedStateBackend.db.put(handle, rocksDBKeyedStateBackend.writeOptions, key, value); } } } @@ -1091,6 +1091,7 @@ private void restoreKeyGroupsShardWithTemporaryHelperInstance( if (stateBackend.keyGroupRange.contains(keyGroup)) { stateBackend.db.put(targetColumnFamilyHandle, + stateBackend.writeOptions, iterator.key(), iterator.value()); } From 50dbac885b790de41e828b63f2585599f6381b5b Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 2 Mar 2018 14:38:20 +0100 Subject: [PATCH 065/268] [FLINK-8517] Fix missing synchronization in TaskEventDispatcher (cherry picked from commit d85fe58) --- .../flink/runtime/io/network/TaskEventDispatcher.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java index 1ec4ade85ad43..c9de902e63aed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java @@ -102,7 +102,10 @@ public void subscribeToEvent( checkNotNull(eventListener); checkNotNull(eventType); - TaskEventHandler taskEventHandler = registeredHandlers.get(partitionId); + TaskEventHandler taskEventHandler; + synchronized (registeredHandlers) { + taskEventHandler = registeredHandlers.get(partitionId); + } if (taskEventHandler == null) { throw new IllegalStateException( "Partition " + partitionId + " not registered at task event dispatcher."); @@ -123,7 +126,10 @@ public boolean publish(ResultPartitionID partitionId, TaskEvent event) { checkNotNull(partitionId); checkNotNull(event); - TaskEventHandler taskEventHandler = registeredHandlers.get(partitionId); + TaskEventHandler taskEventHandler; + synchronized (registeredHandlers) { + taskEventHandler = registeredHandlers.get(partitionId); + } if (taskEventHandler != null) { taskEventHandler.publish(event); From f8681a9d7732f6f36483e26ec68624809f6cd4b1 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 2 Mar 2018 17:46:56 +0100 Subject: [PATCH 066/268] [FLINK-8807] Fix ZookeeperCompleted checkpoint store can get stuck in infinite loop Before, CompletedCheckpoint did not have proper equals()/hashCode(), which meant that the fixpoint condition in ZooKeeperCompletedCheckpointStore would never hold if at least on checkpoint became unreadable. We now compare the interesting fields of the checkpoints manually and extended the test to properly create new CompletedCheckpoints. Before, we were reusing the same CompletedCheckpoint instances, meaning that Objects.equals()/hashCode() would make the test succeed. --- .../checkpoint/CompletedCheckpoint.java | 26 ++++++++++ .../ZooKeeperCompletedCheckpointStore.java | 2 +- ...ZooKeeperCompletedCheckpointStoreTest.java | 51 ++++++++++--------- 3 files changed, 54 insertions(+), 25 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index df8c233ada527..58424272bbc8d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; @@ -37,7 +38,9 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -283,6 +286,29 @@ private void doDiscard() throws Exception { // Miscellaneous // ------------------------------------------------------------------------ + public static boolean checkpointsMatch( + Collection first, + Collection second) { + + Set> firstInterestingFields = + new HashSet<>(); + + for (CompletedCheckpoint checkpoint : first) { + firstInterestingFields.add( + new Tuple2<>(checkpoint.getCheckpointID(), checkpoint.getJobId())); + } + + Set> secondInterestingFields = + new HashSet<>(); + + for (CompletedCheckpoint checkpoint : second) { + secondInterestingFields.add( + new Tuple2<>(checkpoint.getCheckpointID(), checkpoint.getJobId())); + } + + return firstInterestingFields.equals(secondInterestingFields); + } + /** * Sets the callback for tracking when this checkpoint is discarded. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java index 73598e628b657..0cbd4fb6c9e9f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java @@ -199,7 +199,7 @@ public void recover() throws Exception { } } while (retrievedCheckpoints.size() != numberOfInitialCheckpoints && - !lastTryRetrievedCheckpoints.equals(retrievedCheckpoints)); + !CompletedCheckpoint.checkpointsMatch(lastTryRetrievedCheckpoints, retrievedCheckpoints)); // Clear local handles in order to prevent duplicates on // recovery. The local handles should reflect the state diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java index a54432788be2e..08c73bae90045 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java @@ -83,31 +83,16 @@ public void testPathConversion() { /** * Tests that the completed checkpoint store can retrieve all checkpoints stored in ZooKeeper * and ignores those which cannot be retrieved via their state handles. + * + *

    We have a timeout in case the ZooKeeper store get's into a deadlock/livelock situation. */ - @Test + @Test(timeout = 50000) public void testCheckpointRecovery() throws Exception { + final JobID jobID = new JobID(); + final long checkpoint1Id = 1L; + final long checkpoint2Id = 2; final List, String>> checkpointsInZooKeeper = new ArrayList<>(4); - final CompletedCheckpoint completedCheckpoint1 = new CompletedCheckpoint( - new JobID(), - 1L, - 1L, - 1L, - new HashMap<>(), - null, - CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), - new TestCompletedCheckpointStorageLocation()); - - final CompletedCheckpoint completedCheckpoint2 = new CompletedCheckpoint( - new JobID(), - 2L, - 2L, - 2L, - new HashMap<>(), - null, - CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), - new TestCompletedCheckpointStorageLocation()); - final Collection expectedCheckpointIds = new HashSet<>(2); expectedCheckpointIds.add(1L); expectedCheckpointIds.add(2L); @@ -116,10 +101,28 @@ public void testCheckpointRecovery() throws Exception { when(failingRetrievableStateHandle.retrieveState()).thenThrow(new IOException("Test exception")); final RetrievableStateHandle retrievableStateHandle1 = mock(RetrievableStateHandle.class); - when(retrievableStateHandle1.retrieveState()).thenReturn(completedCheckpoint1); + when(retrievableStateHandle1.retrieveState()).then( + (invocation) -> new CompletedCheckpoint( + jobID, + checkpoint1Id, + 1L, + 1L, + new HashMap<>(), + null, + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), + new TestCompletedCheckpointStorageLocation())); final RetrievableStateHandle retrievableStateHandle2 = mock(RetrievableStateHandle.class); - when(retrievableStateHandle2.retrieveState()).thenReturn(completedCheckpoint2); + when(retrievableStateHandle2.retrieveState()).then( + (invocation -> new CompletedCheckpoint( + jobID, + checkpoint2Id, + 2L, + 2L, + new HashMap<>(), + null, + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION), + new TestCompletedCheckpointStorageLocation()))); checkpointsInZooKeeper.add(Tuple2.of(retrievableStateHandle1, "/foobar1")); checkpointsInZooKeeper.add(Tuple2.of(failingRetrievableStateHandle, "/failing1")); @@ -185,7 +188,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { // check that we return the latest retrievable checkpoint // this should remove the latest checkpoint because it is broken - assertEquals(completedCheckpoint2.getCheckpointID(), latestCompletedCheckpoint.getCheckpointID()); + assertEquals(checkpoint2Id, latestCompletedCheckpoint.getCheckpointID()); // this should remove the second broken checkpoint because we're iterating over all checkpoints List completedCheckpoints = zooKeeperCompletedCheckpointStore.getAllCheckpoints(); From 1602b63ce34f7b1f9650aed4a8097e8c552b36d9 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 5 Mar 2018 15:45:36 +0100 Subject: [PATCH 067/268] Fix checkstyle in ZooKeeperCompletedCheckpointStoreTest --- ...ZooKeeperCompletedCheckpointStoreTest.java | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java index 08c73bae90045..0384733fdb1cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java @@ -33,10 +33,8 @@ import org.apache.curator.framework.api.CuratorEventType; import org.apache.curator.framework.api.ErrorListenerPathable; import org.apache.curator.utils.EnsurePath; - import org.junit.Test; import org.junit.runner.RunWith; - import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -67,6 +65,9 @@ import static org.powermock.api.mockito.PowerMockito.doThrow; import static org.powermock.api.mockito.PowerMockito.whenNew; +/** + * Tests for {@link ZooKeeperCompletedCheckpointStore}. + */ @RunWith(PowerMockRunner.class) @PrepareForTest(ZooKeeperCompletedCheckpointStore.class) public class ZooKeeperCompletedCheckpointStoreTest extends TestLogger { @@ -209,7 +210,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { // are subsumed should they be discarded. verify(failingRetrievableStateHandle, never()).discardState(); } - + /** * Tests that the checkpoint does not exist in the store when we fail to add * it into the store (i.e., there exists an exception thrown by the method). @@ -218,29 +219,29 @@ public Void answer(InvocationOnMock invocation) throws Throwable { public void testAddCheckpointWithFailedRemove() throws Exception { final CuratorFramework client = mock(CuratorFramework.class, Mockito.RETURNS_DEEP_STUBS); final RetrievableStateStorageHelper storageHelperMock = mock(RetrievableStateStorageHelper.class); - - ZooKeeperStateHandleStore zookeeperStateHandleStoreMock = + + ZooKeeperStateHandleStore zookeeperStateHandleStoreMock = spy(new ZooKeeperStateHandleStore<>(client, storageHelperMock, Executors.directExecutor())); whenNew(ZooKeeperStateHandleStore.class).withAnyArguments().thenReturn(zookeeperStateHandleStoreMock); - + doAnswer(new Answer>() { @Override public RetrievableStateHandle answer(InvocationOnMock invocationOnMock) throws Throwable { - CompletedCheckpoint checkpoint = (CompletedCheckpoint)invocationOnMock.getArguments()[1]; - + CompletedCheckpoint checkpoint = (CompletedCheckpoint) invocationOnMock.getArguments()[1]; + RetrievableStateHandle retrievableStateHandle = mock(RetrievableStateHandle.class); when(retrievableStateHandle.retrieveState()).thenReturn(checkpoint); - + return retrievableStateHandle; } }).when(zookeeperStateHandleStoreMock).addAndLock(anyString(), any(CompletedCheckpoint.class)); - + doThrow(new Exception()).when(zookeeperStateHandleStoreMock).releaseAndTryRemove(anyString(), any(ZooKeeperStateHandleStore.RemoveCallback.class)); - + final int numCheckpointsToRetain = 1; final String checkpointsPath = "foobar"; final RetrievableStateStorageHelper stateSotrage = mock(RetrievableStateStorageHelper.class); - + ZooKeeperCompletedCheckpointStore zooKeeperCompletedCheckpointStore = new ZooKeeperCompletedCheckpointStore( numCheckpointsToRetain, client, @@ -252,10 +253,10 @@ public RetrievableStateHandle answer(InvocationOnMock invoc CompletedCheckpoint checkpointToAdd = mock(CompletedCheckpoint.class); doReturn(i).when(checkpointToAdd).getCheckpointID(); doReturn(Collections.emptyMap()).when(checkpointToAdd).getOperatorStates(); - + try { zooKeeperCompletedCheckpointStore.addCheckpoint(checkpointToAdd); - + // The checkpoint should be in the store if we successfully add it into the store. List addedCheckpoints = zooKeeperCompletedCheckpointStore.getAllCheckpoints(); assertTrue(addedCheckpoints.contains(checkpointToAdd)); From f34bd8ab4ed63690a8b0f228bd8d0cf421147333 Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Wed, 28 Feb 2018 10:51:42 +0800 Subject: [PATCH 068/268] [hotfix] Fix javadoc link in ClusterClient#triggerSavepoint This closes #5592. --- .../java/org/apache/flink/client/program/ClusterClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index 7817f8f85d079..a4880db636930 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -663,7 +663,7 @@ public void stop(final JobID jobId) throws Exception { /** * Triggers a savepoint for the job identified by the job id. The savepoint will be written to the given savepoint - * directory, or {@link org.apache.flink.configuration.CoreOptions#SAVEPOINT_DIRECTORY} if it is null. + * directory, or {@link org.apache.flink.configuration.CheckpointingOptions#SAVEPOINT_DIRECTORY} if it is null. * * @param jobId job id * @param savepointDirectory directory the savepoint should be written to From 2749175b2e5c5283f0fb173406656e59c8373bc9 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 1 Mar 2018 10:40:20 +0100 Subject: [PATCH 069/268] [hotfix][docs] Drop the incorrect parallel remark in windowAll This closes #5607. --- .../streaming/api/datastream/DataStream.java | 25 ++++++++----------- 1 file changed, 10 insertions(+), 15 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index a357eda118457..7fecdb051954b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -717,9 +717,8 @@ public JoinedStreams join(DataStream otherStream) { * {@code .window(TumblingProcessingTimeWindows.of(size))} depending on the time characteristic * set using * - *

    Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

    Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)} * @@ -741,9 +740,8 @@ public AllWindowedStream timeWindowAll(Time size) { * set using * {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStreamTimeCharacteristic(org.apache.flink.streaming.api.TimeCharacteristic)} * - *

    Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

    Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * @param size The size of the window. */ @@ -758,9 +756,8 @@ public AllWindowedStream timeWindowAll(Time size, Time slide) { /** * Windows this {@code DataStream} into tumbling count windows. * - *

    Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

    Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * @param size The size of the windows in number of elements. */ @@ -771,9 +768,8 @@ public AllWindowedStream countWindowAll(long size) { /** * Windows this {@code DataStream} into sliding count windows. * - *

    Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

    Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * @param size The size of the windows in number of elements. * @param slide The slide interval in number of elements. @@ -794,9 +790,8 @@ public AllWindowedStream countWindowAll(long size, long slide) * when windows are evaluated. However, {@code WindowAssigners} have a default {@code Trigger} * that is used if a {@code Trigger} is not specified. * - *

    Note: This operation can be inherently non-parallel since all elements have to pass through - * the same operator instance. (Only for special cases, such as aligned time windows is - * it possible to perform this operation in parallel). + *

    Note: This operation is inherently non-parallel since all elements have to pass through + * the same operator instance. * * @param assigner The {@code WindowAssigner} that assigns elements to windows. * @return The trigger windows data stream. From 9bd4a408384f841cbfed2e4c941767999e0fcbaf Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Feb 2018 16:04:57 +0100 Subject: [PATCH 070/268] [hotfix][REST] Fix CONTENT_TYPE header This closes #5590. --- .../flink/runtime/webmonitor/WebRuntimeMonitorITCase.java | 2 +- .../apache/flink/runtime/rest/handler/util/HandlerUtils.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java index e6cfdda968c7c..d4fe93af24dca 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitorITCase.java @@ -334,7 +334,7 @@ public void testLeaderNotAvailable() throws Exception { HttpTestClient.SimpleHttpResponse response = client.getNextResponse(); assertEquals(HttpResponseStatus.SERVICE_UNAVAILABLE, response.getStatus()); - assertEquals(MimeTypes.getMimeTypeForExtension("json"), response.getType()); + assertEquals("application/json; charset=UTF-8", response.getType()); assertTrue(response.getContent().contains("refresh")); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java index 604c0b891f345..a69f4aaf4576b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.runtime.rest.messages.ErrorResponseBody; import org.apache.flink.runtime.rest.messages.ResponseBody; +import org.apache.flink.runtime.rest.util.RestConstants; import org.apache.flink.runtime.rest.util.RestMapperUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -149,7 +150,7 @@ public static void sendResponse( @Nonnull Map headers) { HttpResponse response = new DefaultHttpResponse(HTTP_1_1, statusCode); - response.headers().set(CONTENT_TYPE, "application/json"); + response.headers().set(CONTENT_TYPE, RestConstants.REST_CONTENT_TYPE); for (Map.Entry headerEntry : headers.entrySet()) { response.headers().set(headerEntry.getKey(), headerEntry.getValue()); From e19b9fd9debcc56747b1ba364cbe1d5837274d86 Mon Sep 17 00:00:00 2001 From: Ken Krugler Date: Sun, 4 Mar 2018 09:27:11 -0800 Subject: [PATCH 071/268] [FLINK-8849][docs] Fix links to chaining docs This closes #5630. --- docs/concepts/runtime.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/concepts/runtime.md b/docs/concepts/runtime.md index cb6d58f0c1e97..1c7c2816a0e14 100644 --- a/docs/concepts/runtime.md +++ b/docs/concepts/runtime.md @@ -31,7 +31,7 @@ under the License. For distributed execution, Flink *chains* operator subtasks together into *tasks*. Each task is executed by one thread. Chaining operators together into tasks is a useful optimization: it reduces the overhead of thread-to-thread handover and buffering, and increases overall throughput while decreasing latency. -The chaining behavior can be configured; see the [chaining docs](../dev/datastream_api.html#task-chaining-and-resource-groups) for details. +The chaining behavior can be configured; see the [chaining docs](../dev/stream/operators/#task-chaining-and-resource-groups) for details. The sample dataflow in the figure below is executed with five subtasks, and hence with five parallel threads. @@ -98,7 +98,7 @@ job. Allowing this *slot sharing* has two main benefits: TaskManagers with shared Task Slots -The APIs also include a *[resource group](../dev/datastream_api.html#task-chaining-and-resource-groups)* mechanism which can be used to prevent undesirable slot sharing. +The APIs also include a *[resource group](../dev/stream/operators/#task-chaining-and-resource-groups)* mechanism which can be used to prevent undesirable slot sharing. As a rule-of-thumb, a good default number of task slots would be the number of CPU cores. With hyper-threading, each slot then takes 2 or more hardware thread contexts. From 302781dfe5d9e0a6b9246732ac3227db2ced64f6 Mon Sep 17 00:00:00 2001 From: neoremind Date: Mon, 5 Mar 2018 16:50:37 +0800 Subject: [PATCH 072/268] [FLINK-8857][hbase] Remove redundant execute() call in hbase example This closes #5633. --- .../apache/flink/addons/hbase/example/HBaseReadExample.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java index 817ae090c4ef0..8475fb81ec0dd 100644 --- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java +++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/example/HBaseReadExample.java @@ -86,9 +86,6 @@ public boolean filter(Tuple2 t) throws Exception { hbaseDs.print(); - // kick off execution. - env.execute(); - } } From c401edf84f6690b46f37e61aad988eb53787a61f Mon Sep 17 00:00:00 2001 From: Stephen Parente Date: Fri, 2 Mar 2018 14:20:10 -0800 Subject: [PATCH 073/268] [hotfix][docs] Remove reference to CheckpointedRestoring This closes #5627. --- docs/dev/stream/state/state.md | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/dev/stream/state/state.md b/docs/dev/stream/state/state.md index 3ea9b7a7b0e93..a26e24d87027a 100644 --- a/docs/dev/stream/state/state.md +++ b/docs/dev/stream/state/state.md @@ -386,8 +386,7 @@ public class BufferingSink {% highlight scala %} class BufferingSink(threshold: Int = 0) extends SinkFunction[(String, Int)] - with CheckpointedFunction - with CheckpointedRestoring[List[(String, Int)]] { + with CheckpointedFunction { @transient private var checkpointedState: ListState[(String, Int)] = _ @@ -426,9 +425,6 @@ class BufferingSink(threshold: Int = 0) } } - override def restoreState(state: List[(String, Int)]): Unit = { - bufferedElements ++= state - } } {% endhighlight %} From d385e094c8559a2d144bc95f4449c24a696352ab Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Thu, 15 Feb 2018 21:37:44 +0100 Subject: [PATCH 074/268] [FLINK-8560] Add KeyedProcessFunction exposing key in onTimer(). --- docs/dev/stream/operators/process_function.md | 29 +- ...yedProcessOperatorWithWatermarkDelay.scala | 6 +- .../harness/NonWindowHarnessTest.scala | 6 +- .../harness/OverWindowHarnessTest.scala | 16 +- .../SortProcessFunctionHarnessTest.scala | 6 +- .../streaming/api/datastream/KeyedStream.java | 72 ++- .../api/functions/KeyedProcessFunction.java | 130 +++++ .../api/operators/KeyedProcessOperator.java | 46 +- .../operators/LegacyKeyedProcessOperator.java | 178 +++++++ .../flink/streaming/api/DataStreamTest.java | 43 +- .../operators/KeyedProcessOperatorTest.java | 82 +-- .../LegacyKeyedProcessOperatorTest.java | 483 ++++++++++++++++++ .../streaming/api/scala/KeyedStream.scala | 37 +- .../streaming/api/scala/DataStreamTest.scala | 41 +- 14 files changed, 1078 insertions(+), 97 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/KeyedProcessFunction.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LegacyKeyedProcessOperator.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LegacyKeyedProcessOperatorTest.java diff --git a/docs/dev/stream/operators/process_function.md b/docs/dev/stream/operators/process_function.md index a52c5bfeb4858..d96798323ceed 100644 --- a/docs/dev/stream/operators/process_function.md +++ b/docs/dev/stream/operators/process_function.md @@ -242,4 +242,31 @@ class CountWithTimeoutFunction extends ProcessFunction[(String, String), (String the current processing time as event-time timestamp. This behavior is very subtle and might not be noticed by users. Well, it's harmful because processing-time timestamps are indeterministic and not aligned with watermarks. Besides, user-implemented logic depends on this wrong timestamp highly likely is unintendedly faulty. So we've decided to fix it. Upon upgrading to 1.4.0, Flink jobs -that are using this incorrect event-time timestamp will fail, and users should adapt their jobs to the correct logic. \ No newline at end of file +that are using this incorrect event-time timestamp will fail, and users should adapt their jobs to the correct logic. + +## The KeyedProcessFunction + +`KeyedProcessFunction`, as an extension of `ProcessFunction`, gives access to the key of timers in its `onTimer(...)` +method. + +

    +
    +{% highlight java %} +@Override +public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + K key = ctx.getCurrentKey(); + // ... +} + +{% endhighlight %} +
    + +
    +{% highlight scala %} +override def onTimer(timestamp: Long, ctx: OnTimerContext, out: Collector[OUT]): Unit = { + var key = ctx.getCurrentKey + // ... +} +{% endhighlight %} +
    +
    \ No newline at end of file diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala index 74b4773005a65..f63bdb5acd663 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala @@ -19,16 +19,16 @@ package org.apache.flink.table.runtime.operators import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.streaming.api.operators.KeyedProcessOperator +import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator import org.apache.flink.streaming.api.watermark.Watermark /** - * A [[KeyedProcessOperator]] that supports holding back watermarks with a static delay. + * A [[LegacyKeyedProcessOperator]] that supports holding back watermarks with a static delay. */ class KeyedProcessOperatorWithWatermarkDelay[KEY, IN, OUT]( private val function: ProcessFunction[IN, OUT], private var watermarkDelay: Long = 0L) - extends KeyedProcessOperator[KEY, IN, OUT](function) { + extends LegacyKeyedProcessOperator[KEY, IN, OUT](function) { /** emits watermark without delay */ def emitWithoutDelay(mark: Watermark): Unit = output.emitWatermark(mark) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala index ad507618abf5e..5c31cb246306a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/NonWindowHarnessTest.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentLinkedQueue import org.apache.flink.api.common.time.Time import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.streaming.api.operators.KeyedProcessOperator +import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.runtime.aggregate._ @@ -39,7 +39,7 @@ class NonWindowHarnessTest extends HarnessTestBase { @Test def testNonWindow(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new GroupAggProcessFunction( genSumAggFunction, sumAggregationStateType, @@ -99,7 +99,7 @@ class NonWindowHarnessTest extends HarnessTestBase { @Test def testNonWindowWithRetract(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new GroupAggProcessFunction( genSumAggFunction, sumAggregationStateType, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala index def1972866a07..6f6fc0edb1324 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala @@ -23,7 +23,7 @@ import java.util.concurrent.ConcurrentLinkedQueue import org.apache.flink.api.common.time.Time import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.streaming.api.operators.KeyedProcessOperator +import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.runtime.aggregate._ @@ -40,7 +40,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ @Test def testProcTimeBoundedRowsOver(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new ProcTimeBoundedRowsOver( genMinMaxAggFunction, 2, @@ -141,7 +141,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ @Test def testProcTimeBoundedRangeOver(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new ProcTimeBoundedRangeOver( genMinMaxAggFunction, 4000, @@ -250,7 +250,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ @Test def testProcTimeUnboundedOver(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new ProcTimeUnboundedOver( genMinMaxAggFunction, minMaxAggregationStateType, @@ -342,7 +342,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ @Test def testRowTimeBoundedRangeOver(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new RowTimeBoundedRangeOver( genMinMaxAggFunction, minMaxAggregationStateType, @@ -492,7 +492,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ @Test def testRowTimeBoundedRowsOver(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new RowTimeBoundedRowsOver( genMinMaxAggFunction, minMaxAggregationStateType, @@ -640,7 +640,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ @Test def testRowTimeUnboundedRangeOver(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new RowTimeUnboundedRangeOver( genMinMaxAggFunction, minMaxAggregationStateType, @@ -776,7 +776,7 @@ class OverWindowHarnessTest extends HarnessTestBase{ @Test def testRowTimeUnboundedRowsOver(): Unit = { - val processFunction = new KeyedProcessOperator[String, CRow, CRow]( + val processFunction = new LegacyKeyedProcessOperator[String, CRow, CRow]( new RowTimeUnboundedRowsOver( genMinMaxAggFunction, minMaxAggregationStateType, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala index 9490039137822..457bde2ef7aa7 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/SortProcessFunctionHarnessTest.scala @@ -28,7 +28,7 @@ import org.apache.flink.api.java.functions.KeySelector import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.java.typeutils.runtime.RowComparator import org.apache.flink.streaming.api.TimeCharacteristic -import org.apache.flink.streaming.api.operators.KeyedProcessOperator +import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.runtime.streamrecord.StreamRecord import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, TestHarnessUtil} @@ -71,7 +71,7 @@ class SortProcessFunctionHarnessTest { val inputCRowType = CRowTypeInfo(rT) - val processFunction = new KeyedProcessOperator[Integer,CRow,CRow]( + val processFunction = new LegacyKeyedProcessOperator[Integer,CRow,CRow]( new ProcTimeSortProcessFunction( inputCRowType, collectionRowComparator)) @@ -170,7 +170,7 @@ class SortProcessFunctionHarnessTest { val inputCRowType = CRowTypeInfo(rT) - val processFunction = new KeyedProcessOperator[Integer,CRow,CRow]( + val processFunction = new LegacyKeyedProcessOperator[Integer,CRow,CRow]( new RowTimeSortProcessFunction( inputCRowType, 4, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java index 7beaa0369d00e..a948ae20d8cb4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java @@ -37,6 +37,7 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfoBase; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction; import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator; @@ -46,6 +47,7 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.graph.StreamGraphGenerator; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.StreamGroupedFold; import org.apache.flink.streaming.api.operators.StreamGroupedReduce; @@ -272,8 +274,7 @@ public DataStreamSink addSink(SinkFunction sinkFunction) { } /** - * Applies the given {@link ProcessFunction} on the input stream, thereby - * creating a transformed output stream. + * Applies the given {@link ProcessFunction} on the input stream, thereby creating a transformed output stream. * *

    The function will be called for every element in the input streams and can produce zero * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} @@ -286,7 +287,10 @@ public DataStreamSink addSink(SinkFunction sinkFunction) { * @param The type of elements emitted by the {@code ProcessFunction}. * * @return The transformed {@link DataStream}. + * + * @deprecated Use {@link KeyedStream#process(KeyedProcessFunction)} */ + @Deprecated @Override @PublicEvolving public SingleOutputStreamOperator process(ProcessFunction processFunction) { @@ -306,8 +310,7 @@ public SingleOutputStreamOperator process(ProcessFunction processFu } /** - * Applies the given {@link ProcessFunction} on the input stream, thereby - * creating a transformed output stream. + * Applies the given {@link ProcessFunction} on the input stream, thereby creating a transformed output stream. * *

    The function will be called for every element in the input streams and can produce zero * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} @@ -321,19 +324,76 @@ public SingleOutputStreamOperator process(ProcessFunction processFu * @param The type of elements emitted by the {@code ProcessFunction}. * * @return The transformed {@link DataStream}. + * + * @deprecated Use {@link KeyedStream#process(KeyedProcessFunction, TypeInformation)} */ + @Deprecated @Override @Internal public SingleOutputStreamOperator process( ProcessFunction processFunction, TypeInformation outputType) { - KeyedProcessOperator operator = - new KeyedProcessOperator<>(clean(processFunction)); + LegacyKeyedProcessOperator operator = new LegacyKeyedProcessOperator<>(clean(processFunction)); return transform("Process", outputType, operator); } + /** + * Applies the given {@link KeyedProcessFunction} on the input stream, thereby creating a transformed output stream. + * + *

    The function will be called for every element in the input streams and can produce zero + * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} + * function, this function can also query the time and set timers. When reacting to the firing + * of set timers the function can directly emit elements and/or register yet more timers. + * + * @param keyedProcessFunction The {@link KeyedProcessFunction} that is called for each element in the stream. + * + * @param The type of elements emitted by the {@code KeyedProcessFunction}. + * + * @return The transformed {@link DataStream}. + */ + @PublicEvolving + public SingleOutputStreamOperator process(KeyedProcessFunction keyedProcessFunction) { + + TypeInformation outType = TypeExtractor.getUnaryOperatorReturnType( + keyedProcessFunction, + KeyedProcessFunction.class, + 1, + 2, + TypeExtractor.NO_INDEX, + TypeExtractor.NO_INDEX, + getType(), + Utils.getCallLocationName(), + true); + + return process(keyedProcessFunction, outType); + } + + /** + * Applies the given {@link KeyedProcessFunction} on the input stream, thereby creating a transformed output stream. + * + *

    The function will be called for every element in the input streams and can produce zero + * or more output elements. Contrary to the {@link DataStream#flatMap(FlatMapFunction)} + * function, this function can also query the time and set timers. When reacting to the firing + * of set timers the function can directly emit elements and/or register yet more timers. + * + * @param keyedProcessFunction The {@link KeyedProcessFunction} that is called for each element in the stream. + * + * @param outputType {@link TypeInformation} for the result type of the function. + * + * @param The type of elements emitted by the {@code KeyedProcessFunction}. + * + * @return The transformed {@link DataStream}. + */ + @Internal + public SingleOutputStreamOperator process( + KeyedProcessFunction keyedProcessFunction, + TypeInformation outputType) { + + KeyedProcessOperator operator = new KeyedProcessOperator<>(clean(keyedProcessFunction)); + return transform("KeyedProcess", outputType, operator); + } // ------------------------------------------------------------------------ // Windowing diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/KeyedProcessFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/KeyedProcessFunction.java new file mode 100644 index 0000000000000..a03480bc68259 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/KeyedProcessFunction.java @@ -0,0 +1,130 @@ +/* + * 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.streaming.api.functions; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.streaming.api.TimerService; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; + +/** + * A keyed function that processes elements of a stream. + * + *

    For every element in the input stream {@link #processElement(Object, Context, Collector)} + * is invoked. This can produce zero or more elements as output. Implementations can also + * query the time and set timers through the provided {@link Context}. For firing timers + * {@link #onTimer(long, OnTimerContext, Collector)} will be invoked. This can again produce + * zero or more elements as output and register further timers. + * + *

    NOTE: Access to keyed state and timers (which are also scoped to a key) is only + * available if the {@code KeyedProcessFunction} is applied on a {@code KeyedStream}. + * + *

    NOTE: A {@code KeyedProcessFunction} is always a + * {@link org.apache.flink.api.common.functions.RichFunction}. Therefore, access to the + * {@link org.apache.flink.api.common.functions.RuntimeContext} is always available and setup and + * teardown methods can be implemented. See + * {@link org.apache.flink.api.common.functions.RichFunction#open(org.apache.flink.configuration.Configuration)} + * and {@link org.apache.flink.api.common.functions.RichFunction#close()}. + * + * @param Type of the key. + * @param Type of the input elements. + * @param Type of the output elements. + */ +@PublicEvolving +public abstract class KeyedProcessFunction extends AbstractRichFunction { + + private static final long serialVersionUID = 1L; + + /** + * Process one element from the input stream. + * + *

    This function can output zero or more elements using the {@link Collector} parameter + * and also update internal state or set timers using the {@link Context} parameter. + * + * @param value The input value. + * @param ctx A {@link Context} that allows querying the timestamp of the element and getting + * a {@link TimerService} for registering timers and querying the time. The + * context is only valid during the invocation of this method, do not store it. + * @param out The collector for returning result values. + * + * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation + * to fail and may trigger recovery. + */ + public abstract void processElement(I value, Context ctx, Collector out) throws Exception; + + /** + * Called when a timer set using {@link TimerService} fires. + * + * @param timestamp The timestamp of the firing timer. + * @param ctx An {@link OnTimerContext} that allows querying the timestamp, the {@link TimeDomain}, and the key + * of the firing timer and getting a {@link TimerService} for registering timers and querying the time. + * The context is only valid during the invocation of this method, do not store it. + * @param out The collector for returning result values. + * + * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation + * to fail and may trigger recovery. + */ + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception {} + + /** + * Information available in an invocation of {@link #processElement(Object, Context, Collector)} + * or {@link #onTimer(long, OnTimerContext, Collector)}. + */ + public abstract class Context { + + /** + * Timestamp of the element currently being processed or timestamp of a firing timer. + * + *

    This might be {@code null}, for example if the time characteristic of your program + * is set to {@link org.apache.flink.streaming.api.TimeCharacteristic#ProcessingTime}. + */ + public abstract Long timestamp(); + + /** + * A {@link TimerService} for querying time and registering timers. + */ + public abstract TimerService timerService(); + + /** + * Emits a record to the side output identified by the {@link OutputTag}. + * + * @param outputTag the {@code OutputTag} that identifies the side output to emit to. + * @param value The record to emit. + */ + public abstract void output(OutputTag outputTag, X value); + } + + /** + * Information available in an invocation of {@link #onTimer(long, OnTimerContext, Collector)}. + */ + public abstract class OnTimerContext extends Context { + /** + * The {@link TimeDomain} of the firing timer. + */ + public abstract TimeDomain timeDomain(); + + /** + * Get key of the firing timer. + */ + public abstract K getCurrentKey(); + } + +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/KeyedProcessOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/KeyedProcessOperator.java index 6501a9de7d586..b74fdf3492eec 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/KeyedProcessOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/KeyedProcessOperator.java @@ -23,7 +23,7 @@ import org.apache.flink.streaming.api.SimpleTimerService; import org.apache.flink.streaming.api.TimeDomain; import org.apache.flink.streaming.api.TimerService; -import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.OutputTag; @@ -31,12 +31,11 @@ import static org.apache.flink.util.Preconditions.checkState; /** - * A {@link org.apache.flink.streaming.api.operators.StreamOperator} for executing keyed - * {@link ProcessFunction ProcessFunctions}. + * A {@link StreamOperator} for executing {@link KeyedProcessFunction KeyedProcessFunctions}. */ @Internal public class KeyedProcessOperator - extends AbstractUdfStreamOperator> + extends AbstractUdfStreamOperator> implements OneInputStreamOperator, Triggerable { private static final long serialVersionUID = 1L; @@ -47,7 +46,7 @@ public class KeyedProcessOperator private transient OnTimerContextImpl onTimerContext; - public KeyedProcessOperator(ProcessFunction function) { + public KeyedProcessOperator(KeyedProcessFunction function) { super(function); chainingStrategy = ChainingStrategy.ALWAYS; @@ -70,21 +69,13 @@ public void open() throws Exception { @Override public void onEventTime(InternalTimer timer) throws Exception { collector.setAbsoluteTimestamp(timer.getTimestamp()); - onTimerContext.timeDomain = TimeDomain.EVENT_TIME; - onTimerContext.timer = timer; - userFunction.onTimer(timer.getTimestamp(), onTimerContext, collector); - onTimerContext.timeDomain = null; - onTimerContext.timer = null; + invokeUserFunction(TimeDomain.EVENT_TIME, timer); } @Override public void onProcessingTime(InternalTimer timer) throws Exception { collector.eraseTimestamp(); - onTimerContext.timeDomain = TimeDomain.PROCESSING_TIME; - onTimerContext.timer = timer; - userFunction.onTimer(timer.getTimestamp(), onTimerContext, collector); - onTimerContext.timeDomain = null; - onTimerContext.timer = null; + invokeUserFunction(TimeDomain.PROCESSING_TIME, timer); } @Override @@ -95,13 +86,23 @@ public void processElement(StreamRecord element) throws Exception { context.element = null; } - private class ContextImpl extends ProcessFunction.Context { + private void invokeUserFunction( + TimeDomain timeDomain, + InternalTimer timer) throws Exception { + onTimerContext.timeDomain = timeDomain; + onTimerContext.timer = timer; + userFunction.onTimer(timer.getTimestamp(), onTimerContext, collector); + onTimerContext.timeDomain = null; + onTimerContext.timer = null; + } + + private class ContextImpl extends KeyedProcessFunction.Context { private final TimerService timerService; private StreamRecord element; - ContextImpl(ProcessFunction function, TimerService timerService) { + ContextImpl(KeyedProcessFunction function, TimerService timerService) { function.super(); this.timerService = checkNotNull(timerService); } @@ -132,15 +133,15 @@ public void output(OutputTag outputTag, X value) { } } - private class OnTimerContextImpl extends ProcessFunction.OnTimerContext{ + private class OnTimerContextImpl extends KeyedProcessFunction.OnTimerContext { private final TimerService timerService; private TimeDomain timeDomain; - private InternalTimer timer; + private InternalTimer timer; - OnTimerContextImpl(ProcessFunction function, TimerService timerService) { + OnTimerContextImpl(KeyedProcessFunction function, TimerService timerService) { function.super(); this.timerService = checkNotNull(timerService); } @@ -170,5 +171,10 @@ public TimeDomain timeDomain() { checkState(timeDomain != null); return timeDomain; } + + @Override + public K getCurrentKey() { + return timer.getKey(); + } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LegacyKeyedProcessOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LegacyKeyedProcessOperator.java new file mode 100644 index 0000000000000..8481c4680bed4 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LegacyKeyedProcessOperator.java @@ -0,0 +1,178 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.runtime.state.VoidNamespaceSerializer; +import org.apache.flink.streaming.api.SimpleTimerService; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.streaming.api.TimerService; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.OutputTag; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A {@link StreamOperator} for executing keyed {@link ProcessFunction ProcessFunctions}. + * + * @deprecated Replaced by {@link KeyedProcessOperator} which takes {@code KeyedProcessFunction} + */ +@Deprecated +@Internal +public class LegacyKeyedProcessOperator + extends AbstractUdfStreamOperator> + implements OneInputStreamOperator, Triggerable { + + private static final long serialVersionUID = 1L; + + private transient TimestampedCollector collector; + + private transient ContextImpl context; + + private transient OnTimerContextImpl onTimerContext; + + public LegacyKeyedProcessOperator(ProcessFunction function) { + super(function); + + chainingStrategy = ChainingStrategy.ALWAYS; + } + + @Override + public void open() throws Exception { + super.open(); + collector = new TimestampedCollector<>(output); + + InternalTimerService internalTimerService = + getInternalTimerService("user-timers", VoidNamespaceSerializer.INSTANCE, this); + + TimerService timerService = new SimpleTimerService(internalTimerService); + + context = new ContextImpl(userFunction, timerService); + onTimerContext = new OnTimerContextImpl(userFunction, timerService); + } + + @Override + public void onEventTime(InternalTimer timer) throws Exception { + collector.setAbsoluteTimestamp(timer.getTimestamp()); + invokeUserFunction(TimeDomain.EVENT_TIME, timer); + } + + @Override + public void onProcessingTime(InternalTimer timer) throws Exception { + collector.eraseTimestamp(); + invokeUserFunction(TimeDomain.PROCESSING_TIME, timer); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + collector.setTimestamp(element); + context.element = element; + userFunction.processElement(element.getValue(), context, collector); + context.element = null; + } + + private void invokeUserFunction( + TimeDomain timeDomain, + InternalTimer timer) throws Exception { + onTimerContext.timeDomain = timeDomain; + onTimerContext.timer = timer; + userFunction.onTimer(timer.getTimestamp(), onTimerContext, collector); + onTimerContext.timeDomain = null; + onTimerContext.timer = null; + } + + private class ContextImpl extends ProcessFunction.Context { + + private final TimerService timerService; + + private StreamRecord element; + + ContextImpl(ProcessFunction function, TimerService timerService) { + function.super(); + this.timerService = checkNotNull(timerService); + } + + @Override + public Long timestamp() { + checkState(element != null); + + if (element.hasTimestamp()) { + return element.getTimestamp(); + } else { + return null; + } + } + + @Override + public TimerService timerService() { + return timerService; + } + + @Override + public void output(OutputTag outputTag, X value) { + if (outputTag == null) { + throw new IllegalArgumentException("OutputTag must not be null."); + } + + output.collect(outputTag, new StreamRecord<>(value, element.getTimestamp())); + } + } + + private class OnTimerContextImpl extends ProcessFunction.OnTimerContext{ + + private final TimerService timerService; + + private TimeDomain timeDomain; + + private InternalTimer timer; + + OnTimerContextImpl(ProcessFunction function, TimerService timerService) { + function.super(); + this.timerService = checkNotNull(timerService); + } + + @Override + public Long timestamp() { + checkState(timer != null); + return timer.getTimestamp(); + } + + @Override + public TimerService timerService() { + return timerService; + } + + @Override + public void output(OutputTag outputTag, X value) { + if (outputTag == null) { + throw new IllegalArgumentException("OutputTag must not be null."); + } + + output.collect(outputTag, new StreamRecord<>(value, timer.getTimestamp())); + } + + @Override + public TimeDomain timeDomain() { + checkState(timeDomain != null); + return timeDomain; + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index ec8a134e8248b..4fa3fc84ff2da 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -49,6 +49,7 @@ import org.apache.flink.streaming.api.datastream.SplitStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction; import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; @@ -61,6 +62,7 @@ import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; +import org.apache.flink.streaming.api.operators.LegacyKeyedProcessOperator; import org.apache.flink.streaming.api.operators.ProcessOperator; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; @@ -689,11 +691,11 @@ public CustomPOJO fold(CustomPOJO accumulator, String value) throws Exception { } /** - * Verify that a {@link KeyedStream#process(ProcessFunction)} call is correctly translated to - * an operator. + * Verify that a {@link KeyedStream#process(ProcessFunction)} call is correctly translated to an operator. */ @Test - public void testKeyedProcessTranslation() { + @Deprecated + public void testKeyedStreamProcessTranslation() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource src = env.generateSequence(0, 0); @@ -724,12 +726,43 @@ public void onTimer( processed.addSink(new DiscardingSink()); assertEquals(processFunction, getFunctionForDataStream(processed)); + assertTrue(getOperatorForDataStream(processed) instanceof LegacyKeyedProcessOperator); + } + + /** + * Verify that a {@link KeyedStream#process(KeyedProcessFunction)} call is correctly translated to an operator. + */ + @Test + public void testKeyedStreamKeyedProcessTranslation() { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStreamSource src = env.generateSequence(0, 0); + + KeyedProcessFunction keyedProcessFunction = new KeyedProcessFunction() { + private static final long serialVersionUID = 1L; + + @Override + public void processElement(Long value, Context ctx, Collector out) throws Exception { + // Do nothing + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + // Do nothing + } + }; + + DataStream processed = src + .keyBy(new IdentityKeySelector()) + .process(keyedProcessFunction); + + processed.addSink(new DiscardingSink()); + + assertEquals(keyedProcessFunction, getFunctionForDataStream(processed)); assertTrue(getOperatorForDataStream(processed) instanceof KeyedProcessOperator); } /** - * Verify that a {@link DataStream#process(ProcessFunction)} call is correctly translated to - * an operator. + * Verify that a {@link DataStream#process(ProcessFunction)} call is correctly translated to an operator. */ @Test public void testProcessTranslation() { diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java index e1986f37edee2..c5f478cc8c510 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/KeyedProcessOperatorTest.java @@ -24,7 +24,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.api.TimeDomain; -import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; @@ -111,8 +111,10 @@ public void testTimestampAndProcessingTimeQuerying() throws Exception { @Test public void testEventTimeTimers() throws Exception { + final int expectedKey = 17; + KeyedProcessOperator operator = - new KeyedProcessOperator<>(new TriggeringFlatMapFunction(TimeDomain.EVENT_TIME)); + new KeyedProcessOperator<>(new TriggeringFlatMapFunction(TimeDomain.EVENT_TIME, expectedKey)); OneInputStreamOperatorTestHarness testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); @@ -122,14 +124,14 @@ public void testEventTimeTimers() throws Exception { testHarness.processWatermark(new Watermark(0)); - testHarness.processElement(new StreamRecord<>(17, 42L)); + testHarness.processElement(new StreamRecord<>(expectedKey, 42L)); testHarness.processWatermark(new Watermark(5)); ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); expectedOutput.add(new Watermark(0L)); - expectedOutput.add(new StreamRecord<>(17, 42L)); + expectedOutput.add(new StreamRecord<>(expectedKey, 42L)); expectedOutput.add(new StreamRecord<>(1777, 5L)); expectedOutput.add(new Watermark(5L)); @@ -141,8 +143,10 @@ public void testEventTimeTimers() throws Exception { @Test public void testProcessingTimeTimers() throws Exception { + final int expectedKey = 17; + KeyedProcessOperator operator = - new KeyedProcessOperator<>(new TriggeringFlatMapFunction(TimeDomain.PROCESSING_TIME)); + new KeyedProcessOperator<>(new TriggeringFlatMapFunction(TimeDomain.PROCESSING_TIME, expectedKey)); OneInputStreamOperatorTestHarness testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); @@ -150,13 +154,13 @@ public void testProcessingTimeTimers() throws Exception { testHarness.setup(); testHarness.open(); - testHarness.processElement(new StreamRecord<>(17)); + testHarness.processElement(new StreamRecord<>(expectedKey)); testHarness.setProcessingTime(5); ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); - expectedOutput.add(new StreamRecord<>(17)); + expectedOutput.add(new StreamRecord<>(expectedKey)); expectedOutput.add(new StreamRecord<>(1777)); TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); @@ -243,8 +247,10 @@ public void testProcessingTimeTimerWithState() throws Exception { @Test public void testSnapshotAndRestore() throws Exception { + final int expectedKey = 5; + KeyedProcessOperator operator = - new KeyedProcessOperator<>(new BothTriggeringFlatMapFunction()); + new KeyedProcessOperator<>(new BothTriggeringFlatMapFunction(expectedKey)); OneInputStreamOperatorTestHarness testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); @@ -252,14 +258,14 @@ public void testSnapshotAndRestore() throws Exception { testHarness.setup(); testHarness.open(); - testHarness.processElement(new StreamRecord<>(5, 12L)); + testHarness.processElement(new StreamRecord<>(expectedKey, 12L)); // snapshot and restore from scratch OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); testHarness.close(); - operator = new KeyedProcessOperator<>(new BothTriggeringFlatMapFunction()); + operator = new KeyedProcessOperator<>(new BothTriggeringFlatMapFunction(expectedKey)); testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); @@ -283,8 +289,7 @@ public void testSnapshotAndRestore() throws Exception { @Test public void testNullOutputTagRefusal() throws Exception { - KeyedProcessOperator operator = - new KeyedProcessOperator<>(new NullOutputTagEmittingProcessFunction()); + KeyedProcessOperator operator = new KeyedProcessOperator<>(new NullOutputTagEmittingProcessFunction()); OneInputStreamOperatorTestHarness testHarness = new KeyedOneInputStreamOperatorTestHarness<>( @@ -307,8 +312,7 @@ public void testNullOutputTagRefusal() throws Exception { */ @Test public void testSideOutput() throws Exception { - KeyedProcessOperator operator = - new KeyedProcessOperator<>(new SideOutputProcessFunction()); + KeyedProcessOperator operator = new KeyedProcessOperator<>(new SideOutputProcessFunction()); OneInputStreamOperatorTestHarness testHarness = new KeyedOneInputStreamOperatorTestHarness<>( @@ -346,7 +350,7 @@ public void testSideOutput() throws Exception { testHarness.close(); } - private static class NullOutputTagEmittingProcessFunction extends ProcessFunction { + private static class NullOutputTagEmittingProcessFunction extends KeyedProcessFunction { @Override public void processElement(Integer value, Context ctx, Collector out) throws Exception { @@ -354,7 +358,7 @@ public void processElement(Integer value, Context ctx, Collector out) th } } - private static class SideOutputProcessFunction extends ProcessFunction { + private static class SideOutputProcessFunction extends KeyedProcessFunction { static final OutputTag INTEGER_OUTPUT_TAG = new OutputTag("int-out") {}; static final OutputTag LONG_OUTPUT_TAG = new OutputTag("long-out") {}; @@ -377,19 +381,19 @@ public T getKey(T value) throws Exception { } } - private static class QueryingFlatMapFunction extends ProcessFunction { + private static class QueryingFlatMapFunction extends KeyedProcessFunction { private static final long serialVersionUID = 1L; - private final TimeDomain timeDomain; + private final TimeDomain expectedTimeDomain; public QueryingFlatMapFunction(TimeDomain timeDomain) { - this.timeDomain = timeDomain; + this.expectedTimeDomain = timeDomain; } @Override public void processElement(Integer value, Context ctx, Collector out) throws Exception { - if (timeDomain.equals(TimeDomain.EVENT_TIME)) { + if (expectedTimeDomain.equals(TimeDomain.EVENT_TIME)) { out.collect(value + "TIME:" + ctx.timerService().currentWatermark() + " TS:" + ctx.timestamp()); } else { out.collect(value + "TIME:" + ctx.timerService().currentProcessingTime() + " TS:" + ctx.timestamp()); @@ -401,23 +405,26 @@ public void onTimer( long timestamp, OnTimerContext ctx, Collector out) throws Exception { + // Do nothing } } - private static class TriggeringFlatMapFunction extends ProcessFunction { + private static class TriggeringFlatMapFunction extends KeyedProcessFunction { private static final long serialVersionUID = 1L; - private final TimeDomain timeDomain; + private final TimeDomain expectedTimeDomain; + private final Integer expectedKey; - public TriggeringFlatMapFunction(TimeDomain timeDomain) { - this.timeDomain = timeDomain; + public TriggeringFlatMapFunction(TimeDomain timeDomain, Integer expectedKey) { + this.expectedTimeDomain = timeDomain; + this.expectedKey = expectedKey; } @Override public void processElement(Integer value, Context ctx, Collector out) throws Exception { out.collect(value); - if (timeDomain.equals(TimeDomain.EVENT_TIME)) { + if (expectedTimeDomain.equals(TimeDomain.EVENT_TIME)) { ctx.timerService().registerEventTimeTimer(ctx.timerService().currentWatermark() + 5); } else { ctx.timerService().registerProcessingTimeTimer(ctx.timerService().currentProcessingTime() + 5); @@ -429,30 +436,30 @@ public void onTimer( long timestamp, OnTimerContext ctx, Collector out) throws Exception { - - assertEquals(this.timeDomain, ctx.timeDomain()); + assertEquals(expectedKey, ctx.getCurrentKey()); + assertEquals(expectedTimeDomain, ctx.timeDomain()); out.collect(1777); } } - private static class TriggeringStatefulFlatMapFunction extends ProcessFunction { + private static class TriggeringStatefulFlatMapFunction extends KeyedProcessFunction { private static final long serialVersionUID = 1L; private final ValueStateDescriptor state = new ValueStateDescriptor<>("seen-element", IntSerializer.INSTANCE); - private final TimeDomain timeDomain; + private final TimeDomain expectedTimeDomain; public TriggeringStatefulFlatMapFunction(TimeDomain timeDomain) { - this.timeDomain = timeDomain; + this.expectedTimeDomain = timeDomain; } @Override public void processElement(Integer value, Context ctx, Collector out) throws Exception { out.collect("INPUT:" + value); getRuntimeContext().getState(state).update(value); - if (timeDomain.equals(TimeDomain.EVENT_TIME)) { + if (expectedTimeDomain.equals(TimeDomain.EVENT_TIME)) { ctx.timerService().registerEventTimeTimer(ctx.timerService().currentWatermark() + 5); } else { ctx.timerService().registerProcessingTimeTimer(ctx.timerService().currentProcessingTime() + 5); @@ -464,15 +471,21 @@ public void onTimer( long timestamp, OnTimerContext ctx, Collector out) throws Exception { - assertEquals(this.timeDomain, ctx.timeDomain()); + assertEquals(expectedTimeDomain, ctx.timeDomain()); out.collect("STATE:" + getRuntimeContext().getState(state).value()); } } - private static class BothTriggeringFlatMapFunction extends ProcessFunction { + private static class BothTriggeringFlatMapFunction extends KeyedProcessFunction { private static final long serialVersionUID = 1L; + private final Integer expectedKey; + + public BothTriggeringFlatMapFunction(Integer expectedKey) { + this.expectedKey = expectedKey; + } + @Override public void processElement(Integer value, Context ctx, Collector out) throws Exception { ctx.timerService().registerProcessingTimeTimer(5); @@ -484,6 +497,8 @@ public void onTimer( long timestamp, OnTimerContext ctx, Collector out) throws Exception { + assertEquals(expectedKey, ctx.getCurrentKey()); + if (TimeDomain.EVENT_TIME.equals(ctx.timeDomain())) { out.collect("EVENT:1777"); } else { @@ -491,5 +506,4 @@ public void onTimer( } } } - } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LegacyKeyedProcessOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LegacyKeyedProcessOperatorTest.java new file mode 100644 index 0000000000000..970bb35ba7503 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LegacyKeyedProcessOperatorTest.java @@ -0,0 +1,483 @@ +/* + * 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.streaming.api.operators; + +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.TimeDomain; +import org.apache.flink.streaming.api.functions.ProcessFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.TestHarnessUtil; +import org.apache.flink.util.Collector; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.concurrent.ConcurrentLinkedQueue; + +import static org.junit.Assert.assertEquals; + +/** + * Tests {@link LegacyKeyedProcessOperator}. + */ +@Deprecated +public class LegacyKeyedProcessOperatorTest extends TestLogger { + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testTimestampAndWatermarkQuerying() throws Exception { + + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new QueryingFlatMapFunction(TimeDomain.EVENT_TIME)); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.processWatermark(new Watermark(17)); + testHarness.processElement(new StreamRecord<>(5, 12L)); + + testHarness.processWatermark(new Watermark(42)); + testHarness.processElement(new StreamRecord<>(6, 13L)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new Watermark(17L)); + expectedOutput.add(new StreamRecord<>("5TIME:17 TS:12", 12L)); + expectedOutput.add(new Watermark(42L)); + expectedOutput.add(new StreamRecord<>("6TIME:42 TS:13", 13L)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + @Test + public void testTimestampAndProcessingTimeQuerying() throws Exception { + + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new QueryingFlatMapFunction(TimeDomain.PROCESSING_TIME)); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(17); + testHarness.processElement(new StreamRecord<>(5)); + + testHarness.setProcessingTime(42); + testHarness.processElement(new StreamRecord<>(6)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>("5TIME:17 TS:null")); + expectedOutput.add(new StreamRecord<>("6TIME:42 TS:null")); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + @Test + public void testEventTimeTimers() throws Exception { + + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new TriggeringFlatMapFunction(TimeDomain.EVENT_TIME)); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.processWatermark(new Watermark(0)); + + testHarness.processElement(new StreamRecord<>(17, 42L)); + + testHarness.processWatermark(new Watermark(5)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new Watermark(0L)); + expectedOutput.add(new StreamRecord<>(17, 42L)); + expectedOutput.add(new StreamRecord<>(1777, 5L)); + expectedOutput.add(new Watermark(5L)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + @Test + public void testProcessingTimeTimers() throws Exception { + + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new TriggeringFlatMapFunction(TimeDomain.PROCESSING_TIME)); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(17)); + + testHarness.setProcessingTime(5); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>(17)); + expectedOutput.add(new StreamRecord<>(1777)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + /** + * Verifies that we don't have leakage between different keys. + */ + @Test + public void testEventTimeTimerWithState() throws Exception { + + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new TriggeringStatefulFlatMapFunction(TimeDomain.EVENT_TIME)); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.processWatermark(new Watermark(1)); + testHarness.processElement(new StreamRecord<>(17, 0L)); // should set timer for 6 + + testHarness.processWatermark(new Watermark(2)); + testHarness.processElement(new StreamRecord<>(42, 1L)); // should set timer for 7 + + testHarness.processWatermark(new Watermark(6)); + testHarness.processWatermark(new Watermark(7)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new Watermark(1L)); + expectedOutput.add(new StreamRecord<>("INPUT:17", 0L)); + expectedOutput.add(new Watermark(2L)); + expectedOutput.add(new StreamRecord<>("INPUT:42", 1L)); + expectedOutput.add(new StreamRecord<>("STATE:17", 6L)); + expectedOutput.add(new Watermark(6L)); + expectedOutput.add(new StreamRecord<>("STATE:42", 7L)); + expectedOutput.add(new Watermark(7L)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + /** + * Verifies that we don't have leakage between different keys. + */ + @Test + public void testProcessingTimeTimerWithState() throws Exception { + + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new TriggeringStatefulFlatMapFunction(TimeDomain.PROCESSING_TIME)); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(1); + testHarness.processElement(new StreamRecord<>(17)); // should set timer for 6 + + testHarness.setProcessingTime(2); + testHarness.processElement(new StreamRecord<>(42)); // should set timer for 7 + + testHarness.setProcessingTime(6); + testHarness.setProcessingTime(7); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>("INPUT:17")); + expectedOutput.add(new StreamRecord<>("INPUT:42")); + expectedOutput.add(new StreamRecord<>("STATE:17")); + expectedOutput.add(new StreamRecord<>("STATE:42")); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + @Test + public void testSnapshotAndRestore() throws Exception { + + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new BothTriggeringFlatMapFunction()); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(5, 12L)); + + // snapshot and restore from scratch + OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); + + testHarness.close(); + + operator = new LegacyKeyedProcessOperator<>(new BothTriggeringFlatMapFunction()); + + testHarness = new KeyedOneInputStreamOperatorTestHarness<>(operator, new IdentityKeySelector(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.initializeState(snapshot); + testHarness.open(); + + testHarness.setProcessingTime(5); + testHarness.processWatermark(new Watermark(6)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>("PROC:1777")); + expectedOutput.add(new StreamRecord<>("EVENT:1777", 6L)); + expectedOutput.add(new Watermark(6)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + testHarness.close(); + } + + @Test + public void testNullOutputTagRefusal() throws Exception { + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new NullOutputTagEmittingProcessFunction()); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>( + operator, new IdentityKeySelector<>(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.setProcessingTime(17); + try { + expectedException.expect(IllegalArgumentException.class); + testHarness.processElement(new StreamRecord<>(5)); + } finally { + testHarness.close(); + } + } + + /** + * This also verifies that the timestamps ouf side-emitted records is correct. + */ + @Test + public void testSideOutput() throws Exception { + LegacyKeyedProcessOperator operator = + new LegacyKeyedProcessOperator<>(new SideOutputProcessFunction()); + + OneInputStreamOperatorTestHarness testHarness = + new KeyedOneInputStreamOperatorTestHarness<>( + operator, new IdentityKeySelector<>(), BasicTypeInfo.INT_TYPE_INFO); + + testHarness.setup(); + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(42, 17L /* timestamp */)); + + ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue<>(); + + expectedOutput.add(new StreamRecord<>("IN:42", 17L /* timestamp */)); + + TestHarnessUtil.assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); + + ConcurrentLinkedQueue> expectedIntSideOutput = new ConcurrentLinkedQueue<>(); + expectedIntSideOutput.add(new StreamRecord<>(42, 17L /* timestamp */)); + ConcurrentLinkedQueue> intSideOutput = + testHarness.getSideOutput(SideOutputProcessFunction.INTEGER_OUTPUT_TAG); + TestHarnessUtil.assertOutputEquals( + "Side output was not correct.", + expectedIntSideOutput, + intSideOutput); + + ConcurrentLinkedQueue> expectedLongSideOutput = new ConcurrentLinkedQueue<>(); + expectedLongSideOutput.add(new StreamRecord<>(42L, 17L /* timestamp */)); + ConcurrentLinkedQueue> longSideOutput = + testHarness.getSideOutput(SideOutputProcessFunction.LONG_OUTPUT_TAG); + TestHarnessUtil.assertOutputEquals( + "Side output was not correct.", + expectedLongSideOutput, + longSideOutput); + + testHarness.close(); + } + + private static class NullOutputTagEmittingProcessFunction extends ProcessFunction { + + @Override + public void processElement(Integer value, Context ctx, Collector out) throws Exception { + ctx.output(null, value); + } + } + + private static class SideOutputProcessFunction extends ProcessFunction { + + static final OutputTag INTEGER_OUTPUT_TAG = new OutputTag("int-out") {}; + static final OutputTag LONG_OUTPUT_TAG = new OutputTag("long-out") {}; + + @Override + public void processElement(Integer value, Context ctx, Collector out) throws Exception { + out.collect("IN:" + value); + + ctx.output(INTEGER_OUTPUT_TAG, value); + ctx.output(LONG_OUTPUT_TAG, value.longValue()); + } + } + + private static class IdentityKeySelector implements KeySelector { + private static final long serialVersionUID = 1L; + + @Override + public T getKey(T value) throws Exception { + return value; + } + } + + private static class QueryingFlatMapFunction extends ProcessFunction { + + private static final long serialVersionUID = 1L; + + private final TimeDomain timeDomain; + + public QueryingFlatMapFunction(TimeDomain timeDomain) { + this.timeDomain = timeDomain; + } + + @Override + public void processElement(Integer value, Context ctx, Collector out) throws Exception { + if (timeDomain.equals(TimeDomain.EVENT_TIME)) { + out.collect(value + "TIME:" + ctx.timerService().currentWatermark() + " TS:" + ctx.timestamp()); + } else { + out.collect(value + "TIME:" + ctx.timerService().currentProcessingTime() + " TS:" + ctx.timestamp()); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + // Do nothing + } + } + + private static class TriggeringFlatMapFunction extends ProcessFunction { + + private static final long serialVersionUID = 1L; + + private final TimeDomain timeDomain; + + public TriggeringFlatMapFunction(TimeDomain timeDomain) { + this.timeDomain = timeDomain; + } + + @Override + public void processElement(Integer value, Context ctx, Collector out) throws Exception { + out.collect(value); + if (timeDomain.equals(TimeDomain.EVENT_TIME)) { + ctx.timerService().registerEventTimeTimer(ctx.timerService().currentWatermark() + 5); + } else { + ctx.timerService().registerProcessingTimeTimer(ctx.timerService().currentProcessingTime() + 5); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + assertEquals(this.timeDomain, ctx.timeDomain()); + out.collect(1777); + } + } + + private static class TriggeringStatefulFlatMapFunction extends ProcessFunction { + + private static final long serialVersionUID = 1L; + + private final ValueStateDescriptor state = + new ValueStateDescriptor<>("seen-element", IntSerializer.INSTANCE); + + private final TimeDomain timeDomain; + + public TriggeringStatefulFlatMapFunction(TimeDomain timeDomain) { + this.timeDomain = timeDomain; + } + + @Override + public void processElement(Integer value, Context ctx, Collector out) throws Exception { + out.collect("INPUT:" + value); + getRuntimeContext().getState(state).update(value); + if (timeDomain.equals(TimeDomain.EVENT_TIME)) { + ctx.timerService().registerEventTimeTimer(ctx.timerService().currentWatermark() + 5); + } else { + ctx.timerService().registerProcessingTimeTimer(ctx.timerService().currentProcessingTime() + 5); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + assertEquals(this.timeDomain, ctx.timeDomain()); + out.collect("STATE:" + getRuntimeContext().getState(state).value()); + } + } + + private static class BothTriggeringFlatMapFunction extends ProcessFunction { + + private static final long serialVersionUID = 1L; + + @Override + public void processElement(Integer value, Context ctx, Collector out) throws Exception { + ctx.timerService().registerProcessingTimeTimer(5); + ctx.timerService().registerEventTimeTimer(6); + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + if (TimeDomain.EVENT_TIME.equals(ctx.timeDomain())) { + out.collect("EVENT:1777"); + } else { + out.collect("PROC:1777"); + } + } + } +} diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala index 49bdbd9e70c41..51def984898ee 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala @@ -24,7 +24,7 @@ import org.apache.flink.api.common.state.{FoldingStateDescriptor, ReducingStateD import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.streaming.api.datastream.{QueryableStateStream, DataStream => JavaStream, KeyedStream => KeyedJavaStream, WindowedStream => WindowedJavaStream} -import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.functions.{KeyedProcessFunction, ProcessFunction} import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator} import org.apache.flink.streaming.api.functions.query.{QueryableAppendingStateOperator, QueryableValueStateOperator} @@ -66,9 +66,11 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T] * function, this function can also query the time and set timers. When reacting to the firing * of set timers the function can directly emit elements and/or register yet more timers. * - * @param processFunction The [[ProcessFunction]] that is called for each element - * in the stream. + * @param processFunction The [[ProcessFunction]] that is called for each element in the stream. + * + * @deprecated Use [[KeyedStream#process(KeyedProcessFunction)]] */ + @deprecated("will be removed in a future version") @PublicEvolving override def process[R: TypeInformation]( processFunction: ProcessFunction[T, R]): DataStream[R] = { @@ -79,7 +81,34 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T] asScalaStream(javaStream.process(processFunction, implicitly[TypeInformation[R]])) } - + + /** + * Applies the given [[KeyedProcessFunction]] on the input stream, thereby + * creating a transformed output stream. + * + * The function will be called for every element in the stream and can produce + * zero or more output. The function can also query the time and set timers. When + * reacting to the firing of set timers the function can emit yet more elements. + * + * The function will be called for every element in the input streams and can produce zero + * or more output elements. Contrary to the [[DataStream#flatMap(FlatMapFunction)]] + * function, this function can also query the time and set timers. When reacting to the firing + * of set timers the function can directly emit elements and/or register yet more timers. + * + * @param keyedProcessFunction The [[KeyedProcessFunction]] that is called for each element + * in the stream. + */ + @PublicEvolving + def process[R: TypeInformation]( + keyedProcessFunction: KeyedProcessFunction[K, T, R]): DataStream[R] = { + + if (keyedProcessFunction == null) { + throw new NullPointerException("KeyedProcessFunction must not be null.") + } + + asScalaStream(javaStream.process(keyedProcessFunction, implicitly[TypeInformation[R]])) + } + // ------------------------------------------------------------------------ // Windowing // ------------------------------------------------------------------------ diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala index e2c5b416ec783..51ec5e382307a 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala @@ -23,13 +23,11 @@ import java.lang import org.apache.flink.api.common.functions._ import org.apache.flink.api.java.typeutils.TypeExtractor import org.apache.flink.streaming.api.collector.selector.OutputSelector -import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.functions.{KeyedProcessFunction, ProcessFunction} import org.apache.flink.streaming.api.functions.co.CoMapFunction -import org.apache.flink.streaming.api.functions.sink.DiscardingSink import org.apache.flink.streaming.api.graph.{StreamEdge, StreamGraph} -import org.apache.flink.streaming.api.operators.{AbstractUdfStreamOperator, KeyedProcessOperator, ProcessOperator, StreamOperator} +import org.apache.flink.streaming.api.operators._ import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows -import org.apache.flink.streaming.api.windowing.time.Time import org.apache.flink.streaming.api.windowing.triggers.{CountTrigger, PurgingTrigger} import org.apache.flink.streaming.api.windowing.windows.GlobalWindow import org.apache.flink.streaming.runtime.partitioner._ @@ -430,10 +428,11 @@ class DataStreamTest extends AbstractTestBase { } /** - * Verify that a [[KeyedStream.process()]] call is correctly translated to an operator. + * Verify that a [[KeyedStream.process(ProcessFunction)]] call is correctly + * translated to an operator. */ @Test - def testKeyedProcessTranslation(): Unit = { + def testKeyedStreamProcessTranslation(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment val src = env.generateSequence(0, 0) @@ -448,12 +447,36 @@ class DataStreamTest extends AbstractTestBase { val flatMapped = src.keyBy(x => x).process(processFunction) assert(processFunction == getFunctionForDataStream(flatMapped)) + assert(getOperatorForDataStream(flatMapped).isInstanceOf[LegacyKeyedProcessOperator[_, _, _]]) + } + + /** + * Verify that a [[KeyedStream.process(KeyedProcessFunction)]] call is correctly + * translated to an operator. + */ + @Test + def testKeyedStreamKeyedProcessTranslation(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + + val src = env.generateSequence(0, 0) + + val keyedProcessFunction = new KeyedProcessFunction[Long, Long, Int] { + override def processElement( + value: Long, + ctx: KeyedProcessFunction[Long, Long, Int]#Context, + out: Collector[Int]): Unit = ??? + } + + val flatMapped = src.keyBy(x => x).process(keyedProcessFunction) + + assert(keyedProcessFunction == getFunctionForDataStream(flatMapped)) assert(getOperatorForDataStream(flatMapped).isInstanceOf[KeyedProcessOperator[_, _, _]]) } /** - * Verify that a [[DataStream.process()]] call is correctly translated to an operator. - */ + * Verify that a [[DataStream.process(ProcessFunction)]] call is correctly + * translated to an operator. + */ @Test def testProcessTranslation(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment @@ -473,7 +496,6 @@ class DataStreamTest extends AbstractTestBase { assert(getOperatorForDataStream(flatMapped).isInstanceOf[ProcessOperator[_, _]]) } - @Test def operatorTest() { val env = StreamExecutionEnvironment.getExecutionEnvironment @@ -688,5 +710,4 @@ class DataStreamTest extends AbstractTestBase { m.print() m.getId } - } From 80020cb5866c8bac67a48f89aa481de7de262f83 Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Thu, 15 Feb 2018 21:37:44 +0100 Subject: [PATCH 075/268] [FLINK-8667] Expose key in KeyedBroadcastProcessFunction#onTimer() --- .../co/KeyedBroadcastProcessFunction.java | 5 ++ .../co/CoBroadcastWithKeyedOperator.java | 5 ++ .../flink/streaming/api/DataStreamTest.java | 8 +- .../co/CoBroadcastWithKeyedOperatorTest.java | 83 +++++++++++-------- .../api/scala/BroadcastStateITCase.scala | 14 +++- .../runtime/BroadcastStateITCase.java | 24 +++--- 6 files changed, 86 insertions(+), 53 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/co/KeyedBroadcastProcessFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/co/KeyedBroadcastProcessFunction.java index de9cb324dc307..6e6ae5cb62cf7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/co/KeyedBroadcastProcessFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/co/KeyedBroadcastProcessFunction.java @@ -170,5 +170,10 @@ public abstract class OnTimerContext extends KeyedReadOnlyContext { * event or processing time timer. */ public abstract TimeDomain timeDomain(); + + /** + * Get the key of the firing timer. + */ + public abstract KS getCurrentKey(); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/co/CoBroadcastWithKeyedOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/co/CoBroadcastWithKeyedOperator.java index 2bdb6832b8c2d..871363b68d5e5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/co/CoBroadcastWithKeyedOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/co/CoBroadcastWithKeyedOperator.java @@ -324,6 +324,11 @@ public TimeDomain timeDomain() { return timeDomain; } + @Override + public KS getCurrentKey() { + return timer.getKey(); + } + @Override public TimerService timerService() { return timerService; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java index 4fa3fc84ff2da..632667217633f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java @@ -707,7 +707,7 @@ public void processElement( Long value, Context ctx, Collector out) throws Exception { - + // Do nothing } @Override @@ -715,7 +715,7 @@ public void onTimer( long timestamp, OnTimerContext ctx, Collector out) throws Exception { - + // Do nothing } }; @@ -777,7 +777,7 @@ public void processElement( Long value, Context ctx, Collector out) throws Exception { - + // Do nothing } @Override @@ -785,7 +785,7 @@ public void onTimer( long timestamp, OnTimerContext ctx, Collector out) throws Exception { - + // Do nothing } }; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoBroadcastWithKeyedOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoBroadcastWithKeyedOperatorTest.java index 96607d404d26b..b923b751d33b9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoBroadcastWithKeyedOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoBroadcastWithKeyedOperatorTest.java @@ -38,7 +38,6 @@ import org.apache.flink.util.OutputTag; import org.apache.flink.util.Preconditions; -import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; @@ -54,6 +53,11 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.function.Function; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + /** * Tests for the {@link CoBroadcastWithKeyedOperator}. */ @@ -148,7 +152,7 @@ public void process(String key, ListState state) throws Exception { while (it.hasNext()) { list.add(it.next()); } - Assert.assertEquals(expectedKeyedStates.get(key), list); + assertEquals(expectedKeyedStates.get(key), list); } }); } @@ -161,12 +165,13 @@ public void processElement(String value, KeyedReadOnlyContext ctx, Collector testHarness = getInitializedTestHarness( BasicTypeInfo.STRING_TYPE_INFO, new IdentityKeySelector<>(), - new FunctionWithTimerOnKeyed(41L)) + new FunctionWithTimerOnKeyed(41L, expectedKey)) ) { testHarness.processWatermark1(new Watermark(10L)); testHarness.processWatermark2(new Watermark(10L)); @@ -174,8 +179,8 @@ public void testFunctionWithTimer() throws Exception { testHarness.processWatermark1(new Watermark(40L)); testHarness.processWatermark2(new Watermark(40L)); - testHarness.processElement1(new StreamRecord<>("6", 13L)); - testHarness.processElement1(new StreamRecord<>("6", 15L)); + testHarness.processElement1(new StreamRecord<>(expectedKey, 13L)); + testHarness.processElement1(new StreamRecord<>(expectedKey, 15L)); testHarness.processWatermark1(new Watermark(50L)); testHarness.processWatermark2(new Watermark(50L)); @@ -203,9 +208,11 @@ private static class FunctionWithTimerOnKeyed extends KeyedBroadcastProcessFunct private static final long serialVersionUID = 7496674620398203933L; private final long timerTS; + private final String expectedKey; - FunctionWithTimerOnKeyed(long timerTS) { + FunctionWithTimerOnKeyed(long timerTS, String expectedKey) { this.timerTS = timerTS; + this.expectedKey = expectedKey; } @Override @@ -221,6 +228,7 @@ public void processElement(String value, KeyedReadOnlyContext ctx, Collector out) throws Exception { + assertEquals(expectedKey, ctx.getCurrentKey()); out.collect("TIMER:" + timestamp); } } @@ -293,7 +301,6 @@ public void processElement(String value, KeyedReadOnlyContext ctx, Collector expectedBroadcastState = new HashMap<>(); expectedBroadcastState.put("5.key", 5); expectedBroadcastState.put("34.key", 34); @@ -301,11 +308,13 @@ public void testFunctionWithBroadcastState() throws Exception { expectedBroadcastState.put("12.key", 12); expectedBroadcastState.put("98.key", 98); + final String expectedKey = "trigger"; + try ( TwoInputStreamOperatorTestHarness testHarness = getInitializedTestHarness( BasicTypeInfo.STRING_TYPE_INFO, new IdentityKeySelector<>(), - new FunctionWithBroadcastState("key", expectedBroadcastState, 41L)) + new FunctionWithBroadcastState("key", expectedBroadcastState, 41L, expectedKey)) ) { testHarness.processWatermark1(new Watermark(10L)); testHarness.processWatermark2(new Watermark(10L)); @@ -316,7 +325,7 @@ public void testFunctionWithBroadcastState() throws Exception { testHarness.processElement2(new StreamRecord<>(12, 16L)); testHarness.processElement2(new StreamRecord<>(98, 19L)); - testHarness.processElement1(new StreamRecord<>("trigger", 13L)); + testHarness.processElement1(new StreamRecord<>(expectedKey, 13L)); testHarness.processElement2(new StreamRecord<>(51, 21L)); @@ -324,29 +333,29 @@ public void testFunctionWithBroadcastState() throws Exception { testHarness.processWatermark2(new Watermark(50L)); Queue output = testHarness.getOutput(); - Assert.assertEquals(3L, output.size()); + assertEquals(3L, output.size()); Object firstRawWm = output.poll(); - Assert.assertTrue(firstRawWm instanceof Watermark); + assertTrue(firstRawWm instanceof Watermark); Watermark firstWm = (Watermark) firstRawWm; - Assert.assertEquals(10L, firstWm.getTimestamp()); + assertEquals(10L, firstWm.getTimestamp()); Object rawOutputElem = output.poll(); - Assert.assertTrue(rawOutputElem instanceof StreamRecord); + assertTrue(rawOutputElem instanceof StreamRecord); StreamRecord outputRec = (StreamRecord) rawOutputElem; - Assert.assertTrue(outputRec.getValue() instanceof String); + assertTrue(outputRec.getValue() instanceof String); String outputElem = (String) outputRec.getValue(); expectedBroadcastState.put("51.key", 51); List> expectedEntries = new ArrayList<>(); expectedEntries.addAll(expectedBroadcastState.entrySet()); String expected = "TS:41 " + mapToString(expectedEntries); - Assert.assertEquals(expected, outputElem); + assertEquals(expected, outputElem); Object secondRawWm = output.poll(); - Assert.assertTrue(secondRawWm instanceof Watermark); + assertTrue(secondRawWm instanceof Watermark); Watermark secondWm = (Watermark) secondRawWm; - Assert.assertEquals(50L, secondWm.getTimestamp()); + assertEquals(50L, secondWm.getTimestamp()); } } @@ -357,15 +366,17 @@ private static class FunctionWithBroadcastState extends KeyedBroadcastProcessFun private final String keyPostfix; private final Map expectedBroadcastState; private final long timerTs; + private final String expectedKey; FunctionWithBroadcastState( final String keyPostfix, final Map expectedBroadcastState, - final long timerTs - ) { + final long timerTs, + final String expectedKey) { this.keyPostfix = Preconditions.checkNotNull(keyPostfix); this.expectedBroadcastState = Preconditions.checkNotNull(expectedBroadcastState); this.timerTs = timerTs; + this.expectedKey = expectedKey; } @Override @@ -381,14 +392,14 @@ public void processElement(String value, KeyedReadOnlyContext ctx, Collector> iter = broadcastStateIt.iterator(); for (int i = 0; i < expectedBroadcastState.size(); i++) { - Assert.assertTrue(iter.hasNext()); + assertTrue(iter.hasNext()); Map.Entry entry = iter.next(); - Assert.assertTrue(expectedBroadcastState.containsKey(entry.getKey())); - Assert.assertEquals(expectedBroadcastState.get(entry.getKey()), entry.getValue()); + assertTrue(expectedBroadcastState.containsKey(entry.getKey())); + assertEquals(expectedBroadcastState.get(entry.getKey()), entry.getValue()); } - Assert.assertFalse(iter.hasNext()); + assertFalse(iter.hasNext()); ctx.timerService().registerEventTimeTimer(timerTs); } @@ -401,6 +412,8 @@ public void onTimer(long timestamp, OnTimerContext ctx, Collector out) t while (iter.hasNext()) { map.add(iter.next()); } + + assertEquals(expectedKey, ctx.getCurrentKey()); final String mapToStr = mapToString(map); out.collect("TS:" + timestamp + " " + mapToStr); } @@ -485,22 +498,22 @@ public void testScaleUp() throws Exception { Queue output2 = testHarness2.getOutput(); Queue output3 = testHarness3.getOutput(); - Assert.assertEquals(expected.size(), output1.size()); + assertEquals(expected.size(), output1.size()); for (Object o: output1) { StreamRecord rec = (StreamRecord) o; - Assert.assertTrue(expected.contains(rec.getValue())); + assertTrue(expected.contains(rec.getValue())); } - Assert.assertEquals(expected.size(), output2.size()); + assertEquals(expected.size(), output2.size()); for (Object o: output2) { StreamRecord rec = (StreamRecord) o; - Assert.assertTrue(expected.contains(rec.getValue())); + assertTrue(expected.contains(rec.getValue())); } - Assert.assertEquals(expected.size(), output3.size()); + assertEquals(expected.size(), output3.size()); for (Object o: output3) { StreamRecord rec = (StreamRecord) o; - Assert.assertTrue(expected.contains(rec.getValue())); + assertTrue(expected.contains(rec.getValue())); } } } @@ -583,16 +596,16 @@ public void testScaleDown() throws Exception { Queue output1 = testHarness1.getOutput(); Queue output2 = testHarness2.getOutput(); - Assert.assertEquals(expected.size(), output1.size()); + assertEquals(expected.size(), output1.size()); for (Object o: output1) { StreamRecord rec = (StreamRecord) o; - Assert.assertTrue(expected.contains(rec.getValue())); + assertTrue(expected.contains(rec.getValue())); } - Assert.assertEquals(expected.size(), output2.size()); + assertEquals(expected.size(), output2.size()); for (Object o: output2) { StreamRecord rec = (StreamRecord) o; - Assert.assertTrue(expected.contains(rec.getValue())); + assertTrue(expected.contains(rec.getValue())); } } } @@ -653,12 +666,12 @@ public void processElement(String value, KeyedReadOnlyContext ctx, Collector(5, 12L)); } catch (NullPointerException e) { - Assert.assertEquals("No key set. This method should not be called outside of a keyed context.", e.getMessage()); + assertEquals("No key set. This method should not be called outside of a keyed context.", e.getMessage()); exceptionThrown = true; } if (!exceptionThrown) { - Assert.fail("No exception thrown"); + fail("No exception thrown"); } } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/BroadcastStateITCase.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/BroadcastStateITCase.scala index 6c382d5f4c5d6..55bb3ba420152 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/BroadcastStateITCase.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/BroadcastStateITCase.scala @@ -28,7 +28,7 @@ import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.test.util.AbstractTestBase import org.apache.flink.util.Collector import org.junit.Assert.assertEquals -import org.junit.{Assert, Test} +import org.junit.{Test} /** * ITCase for the [[org.apache.flink.api.common.state.BroadcastState]]. @@ -103,13 +103,19 @@ class TestBroadcastProcessFunction( BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], BasicTypeInfo.STRING_TYPE_INFO) + var timerToExpectedKey = Map[Long, Long]() + var nextTimerTimestamp :Long = expectedTimestamp + @throws[Exception] override def processElement( value: Long, ctx: KeyedBroadcastProcessFunction[Long, Long, String, String]#KeyedReadOnlyContext, out: Collector[String]): Unit = { - ctx.timerService.registerEventTimeTimer(expectedTimestamp) + val currentTime = nextTimerTimestamp + nextTimerTimestamp += 1 + ctx.timerService.registerEventTimeTimer(currentTime) + timerToExpectedKey += (currentTime -> value) } @throws[Exception] @@ -128,6 +134,8 @@ class TestBroadcastProcessFunction( ctx: KeyedBroadcastProcessFunction[Long, Long, String, String]#OnTimerContext, out: Collector[String]): Unit = { + assertEquals(timerToExpectedKey(timestamp), ctx.getCurrentKey) + var map = Map[Long, String]() import scala.collection.JavaConversions._ @@ -137,7 +145,7 @@ class TestBroadcastProcessFunction( map += (entry.getKey -> entry.getValue) } - Assert.assertEquals(expectedBroadcastState, map) + assertEquals(expectedBroadcastState, map) out.collect(timestamp.toString) } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BroadcastStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BroadcastStateITCase.java index 868aca91a9d8f..7ccba3337e6f7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BroadcastStateITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/BroadcastStateITCase.java @@ -32,7 +32,6 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.util.Collector; -import org.junit.Assert; import org.junit.Test; import javax.annotation.Nullable; @@ -40,6 +39,8 @@ import java.util.HashMap; import java.util.Map; +import static org.junit.Assert.assertEquals; + /** * ITCase for the {@link org.apache.flink.api.common.state.BroadcastState}. */ @@ -120,7 +121,7 @@ public void close() throws Exception { super.close(); // make sure that all the timers fired - Assert.assertEquals(expectedOutputCounter, outputCounter); + assertEquals(expectedOutputCounter, outputCounter); } } @@ -145,17 +146,15 @@ private static class TestBroadcastProcessFunction extends KeyedBroadcastProcessF private static final long serialVersionUID = 7616910653561100842L; private final Map expectedState; + private final Map timerToExpectedKey = new HashMap<>(); - private final long timerTimestamp; + private long nextTimerTimestamp; private transient MapStateDescriptor descriptor; - TestBroadcastProcessFunction( - final long timerTS, - final Map expectedBroadcastState - ) { + TestBroadcastProcessFunction(final long initialTimerTimestamp, final Map expectedBroadcastState) { expectedState = expectedBroadcastState; - timerTimestamp = timerTS; + nextTimerTimestamp = initialTimerTimestamp; } @Override @@ -169,7 +168,10 @@ public void open(Configuration parameters) throws Exception { @Override public void processElement(Long value, KeyedReadOnlyContext ctx, Collector out) throws Exception { - ctx.timerService().registerEventTimeTimer(timerTimestamp); + long currentTime = nextTimerTimestamp; + nextTimerTimestamp++; + ctx.timerService().registerEventTimeTimer(currentTime); + timerToExpectedKey.put(currentTime, value); } @Override @@ -180,14 +182,14 @@ public void processBroadcastElement(String value, KeyedContext ctx, Collector out) throws Exception { - Assert.assertEquals(timerTimestamp, timestamp); + assertEquals(timerToExpectedKey.get(timestamp), ctx.getCurrentKey()); Map map = new HashMap<>(); for (Map.Entry entry : ctx.getBroadcastState(descriptor).immutableEntries()) { map.put(entry.getKey(), entry.getValue()); } - Assert.assertEquals(expectedState, map); + assertEquals(expectedState, map); out.collect(Long.toString(timestamp)); } From 969909bff0dd9e2f6394a623c3f7f64fe04b82b0 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 7 Mar 2018 11:58:07 +0100 Subject: [PATCH 076/268] [FLINK-8890] Compare checkpoints with order in CompletedCheckpoint.checkpointsMatch() This method is used, among other things, to check if a list of restored checkpoints is stable after several restore attempts in the ZooKeeper checkpoint store. The order of checkpoints is somewhat important because we want the latest checkpoint to stay the latest checkpoint. --- .../checkpoint/CompletedCheckpoint.java | 12 +- .../checkpoint/CompletedCheckpointTest.java | 123 ++++++++++++++++++ 2 files changed, 129 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index 58424272bbc8d..1932b19925cb4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -38,9 +38,8 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; +import java.util.List; import java.util.Map; -import java.util.Set; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -289,17 +288,18 @@ private void doDiscard() throws Exception { public static boolean checkpointsMatch( Collection first, Collection second) { + if (first.size() != second.size()) { + return false; + } - Set> firstInterestingFields = - new HashSet<>(); + List> firstInterestingFields = new ArrayList<>(first.size()); for (CompletedCheckpoint checkpoint : first) { firstInterestingFields.add( new Tuple2<>(checkpoint.getCheckpointID(), checkpoint.getJobId())); } - Set> secondInterestingFields = - new HashSet<>(); + List> secondInterestingFields = new ArrayList<>(second.size()); for (CompletedCheckpoint checkpoint : second) { secondInterestingFields.add( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java index 69003cd984c2a..5af7c76ec6eff 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java @@ -31,8 +31,10 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; @@ -50,6 +52,127 @@ public class CompletedCheckpointTest { @Rule public final TemporaryFolder tmpFolder = new TemporaryFolder(); + @Test + public void testCompareCheckpointsWithDifferentOrder() { + + CompletedCheckpoint checkpoint1 = new CompletedCheckpoint( + new JobID(), 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + new TestCompletedCheckpointStorageLocation()); + + CompletedCheckpoint checkpoint2 = new CompletedCheckpoint( + new JobID(), 1, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + new TestCompletedCheckpointStorageLocation()); + + List checkpoints1= new ArrayList<>(); + checkpoints1.add(checkpoint1); + checkpoints1.add(checkpoint2); + checkpoints1.add(checkpoint1); + + List checkpoints2 = new ArrayList<>(); + checkpoints2.add(checkpoint2); + checkpoints2.add(checkpoint1); + checkpoints2.add(checkpoint2); + + assertFalse(CompletedCheckpoint.checkpointsMatch(checkpoints1, checkpoints2)); + } + + @Test + public void testCompareCheckpointsWithSameOrder() { + + CompletedCheckpoint checkpoint1 = new CompletedCheckpoint( + new JobID(), 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + new TestCompletedCheckpointStorageLocation()); + + CompletedCheckpoint checkpoint2 = new CompletedCheckpoint( + new JobID(), 1, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + new TestCompletedCheckpointStorageLocation()); + + List checkpoints1= new ArrayList<>(); + checkpoints1.add(checkpoint1); + checkpoints1.add(checkpoint2); + checkpoints1.add(checkpoint1); + + List checkpoints2 = new ArrayList<>(); + checkpoints2.add(checkpoint1); + checkpoints2.add(checkpoint2); + checkpoints2.add(checkpoint1); + + assertTrue(CompletedCheckpoint.checkpointsMatch(checkpoints1, checkpoints2)); + } + + /** + * Verify that both JobID and checkpoint id are taken into account when comparing. + */ + @Test + public void testCompareCheckpointsWithSameJobID() { + JobID jobID = new JobID(); + + CompletedCheckpoint checkpoint1 = new CompletedCheckpoint( + jobID, 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + new TestCompletedCheckpointStorageLocation()); + + CompletedCheckpoint checkpoint2 = new CompletedCheckpoint( + jobID, 1, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + new TestCompletedCheckpointStorageLocation()); + + List checkpoints1= new ArrayList<>(); + checkpoints1.add(checkpoint1); + + List checkpoints2 = new ArrayList<>(); + checkpoints2.add(checkpoint2); + + assertFalse(CompletedCheckpoint.checkpointsMatch(checkpoints1, checkpoints2)); + } + + /** + * Verify that both JobID and checkpoint id are taken into account when comparing. + */ + @Test + public void testCompareCheckpointsWithSameCheckpointId() { + JobID jobID1 = new JobID(); + JobID jobID2 = new JobID(); + + CompletedCheckpoint checkpoint1 = new CompletedCheckpoint( + jobID1, 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + new TestCompletedCheckpointStorageLocation()); + + CompletedCheckpoint checkpoint2 = new CompletedCheckpoint( + jobID2, 0, 0, 1, + new HashMap<>(), + Collections.emptyList(), + CheckpointProperties.forCheckpoint(CheckpointRetentionPolicy.RETAIN_ON_FAILURE), + new TestCompletedCheckpointStorageLocation()); + + List checkpoints1= new ArrayList<>(); + checkpoints1.add(checkpoint1); + + List checkpoints2 = new ArrayList<>(); + checkpoints2.add(checkpoint2); + + assertFalse(CompletedCheckpoint.checkpointsMatch(checkpoints1, checkpoints2)); + } + @Test public void testRegisterStatesAtRegistry() { OperatorState state = mock(OperatorState.class); From af9a68c5af85feb51de5cac735de1473f008f0fe Mon Sep 17 00:00:00 2001 From: davidxdh Date: Fri, 2 Mar 2018 10:55:23 +0800 Subject: [PATCH 077/268] [FLINK-8827] [scripts] When FLINK_CONF_DIR contains spaces, ZooKeeper related scripts fail This closes #5614 --- flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh | 6 +++--- flink-dist/src/main/flink-bin/bin/stop-zookeeper-quorum.sh | 6 +++--- flink-dist/src/main/flink-bin/bin/zookeeper.sh | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh b/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh index cc8917b054737..d5a7593a61f3a 100755 --- a/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh +++ b/flink-dist/src/main/flink-bin/bin/start-zookeeper-quorum.sh @@ -24,8 +24,8 @@ bin=`cd "$bin"; pwd` # Starts a ZooKeeper quorum as configured in $FLINK_CONF/zoo.cfg -ZK_CONF=$FLINK_CONF_DIR/zoo.cfg -if [ ! -f $ZK_CONF ]; then +ZK_CONF="$FLINK_CONF_DIR/zoo.cfg" +if [ ! -f "$ZK_CONF" ]; then echo "[ERROR] No ZooKeeper configuration file found in '$ZK_CONF'." exit 1 fi @@ -43,4 +43,4 @@ while read server ; do else echo "[WARN] Parse error. Skipping config entry '$server'." fi -done < <(grep "^server\." $ZK_CONF) +done < <(grep "^server\." "$ZK_CONF") diff --git a/flink-dist/src/main/flink-bin/bin/stop-zookeeper-quorum.sh b/flink-dist/src/main/flink-bin/bin/stop-zookeeper-quorum.sh index 29ddae4193119..ad79de83a1296 100755 --- a/flink-dist/src/main/flink-bin/bin/stop-zookeeper-quorum.sh +++ b/flink-dist/src/main/flink-bin/bin/stop-zookeeper-quorum.sh @@ -24,8 +24,8 @@ bin=`cd "$bin"; pwd` # Stops a ZooKeeper quorum as configured in $FLINK_CONF/zoo.cfg -ZK_CONF=$FLINK_CONF_DIR/zoo.cfg -if [ ! -f $ZK_CONF ]; then +ZK_CONF="$FLINK_CONF_DIR/zoo.cfg" +if [ ! -f "$ZK_CONF" ]; then echo "[ERROR] No ZooKeeper configuration file found in '$ZK_CONF'." exit 1 fi @@ -43,4 +43,4 @@ while read server ; do else echo "[WARN] Parse error. Skipping config entry '$server'." fi -done < <(grep "^server\." $ZK_CONF) +done < <(grep "^server\." "$ZK_CONF") diff --git a/flink-dist/src/main/flink-bin/bin/zookeeper.sh b/flink-dist/src/main/flink-bin/bin/zookeeper.sh index ca72bb7eae53e..53d709833656c 100755 --- a/flink-dist/src/main/flink-bin/bin/zookeeper.sh +++ b/flink-dist/src/main/flink-bin/bin/zookeeper.sh @@ -33,8 +33,8 @@ bin=`cd "$bin"; pwd` . "$bin"/config.sh -ZK_CONF=$FLINK_CONF_DIR/zoo.cfg -if [ ! -f $ZK_CONF ]; then +ZK_CONF="$FLINK_CONF_DIR/zoo.cfg" +if [ ! -f "$ZK_CONF" ]; then echo "[ERROR] No ZooKeeper configuration file found in '$ZK_CONF'." exit 1 fi From 70347bcc788dea1d4f64539701e60bafcc772946 Mon Sep 17 00:00:00 2001 From: zhangminglei Date: Fri, 2 Mar 2018 20:31:49 +0800 Subject: [PATCH 078/268] [FLINK-8824] [kafka connector] Replace Class.getCanonicalName() with Class.getName() This closes #5620 --- .../streaming/connectors/kafka/FlinkKafkaProducer011.java | 4 ++-- .../streaming/connectors/kafka/FlinkKafkaConsumer09.java | 2 +- .../streaming/connectors/kafka/FlinkKafkaProducerBase.java | 4 ++-- .../connectors/kafka/FlinkKafkaProducerBaseTest.java | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java index 3fe4bc6b015f2..e92f38b3ea063 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011.java @@ -492,13 +492,13 @@ public FlinkKafkaProducer011( // set the producer configuration properties for kafka record key value serializers. if (!producerConfig.containsKey(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG)) { - this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName()); + this.producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); } else { LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); } if (!producerConfig.containsKey(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG)) { - this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getCanonicalName()); + this.producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName()); } else { LOG.warn("Overwriting the '{}' is not recommended", ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); } diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java index 497003293415f..00b7da401e8d1 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer09.java @@ -296,7 +296,7 @@ protected Map fetchOffsetsWithTimestamp(Collection Date: Tue, 6 Mar 2018 18:04:05 +0000 Subject: [PATCH 079/268] [hotfix] [javadoc] Minor javadoc fix in TimestampAssigner.java This closes #5646 Also close unrelated lingering pull request: This closes #5643 --- .../flink/streaming/api/functions/TimestampAssigner.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/TimestampAssigner.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/TimestampAssigner.java index 60debb966d07e..ba7bdd87e16c2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/TimestampAssigner.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/TimestampAssigner.java @@ -40,9 +40,9 @@ public interface TimestampAssigner extends Function { * by ingestion time. If the element did not carry a timestamp before, this value is * {@code Long.MIN_VALUE}. * - * @param element The element that the timestamp is wil be assigned to. + * @param element The element that the timestamp will be assigned to. * @param previousElementTimestamp The previous internal timestamp of the element, - * or a negative value, if no timestamp has been assigned, yet. + * or a negative value, if no timestamp has been assigned yet. * @return The new timestamp. */ long extractTimestamp(T element, long previousElementTimestamp); From b0418b41f8fa02d3217b760c5bdfcdd7efdc1eac Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 4 Mar 2018 12:11:29 +0100 Subject: [PATCH 080/268] [FLINK-8877] [core] Set Kryo trace if Flink log level is TRACE --- .../runtime/kryo/KryoSerializer.java | 14 +++++ .../runtime/kryo/MinlogForwarder.java | 61 +++++++++++++++++++ 2 files changed, 75 insertions(+) create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/MinlogForwarder.java diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java index f60ce460e6d76..06ba906c241b0 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java @@ -74,6 +74,10 @@ public class KryoSerializer extends TypeSerializer { private static final Logger LOG = LoggerFactory.getLogger(KryoSerializer.class); + static { + configureKryoLogging(); + } + // ------------------------------------------------------------------------ private final LinkedHashMap, ExecutionConfig.SerializableSerializer> defaultSerializers; @@ -483,6 +487,16 @@ private static LinkedHashMap buildKryoRegistrations( return kryoRegistrations; } + static void configureKryoLogging() { + // Kryo uses only DEBUG and TRACE levels + // we only forward TRACE level, because even DEBUG levels results in + // a logging for each object, which is infeasible in Flink. + if (LOG.isTraceEnabled()) { + com.esotericsoftware.minlog.Log.setLogger(new MinlogForwarder(LOG)); + com.esotericsoftware.minlog.Log.TRACE(); + } + } + // -------------------------------------------------------------------------------------------- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/MinlogForwarder.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/MinlogForwarder.java new file mode 100644 index 0000000000000..3467923f9fc03 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/MinlogForwarder.java @@ -0,0 +1,61 @@ +/* + * 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.api.java.typeutils.runtime.kryo; + +import org.apache.flink.annotation.Internal; + +import com.esotericsoftware.minlog.Log; +import com.esotericsoftware.minlog.Log.Logger; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An implementation of the Minlog Logger that forwards to slf4j. + */ +@Internal +class MinlogForwarder extends Logger { + + private final org.slf4j.Logger log; + + MinlogForwarder(org.slf4j.Logger log) { + this.log = checkNotNull(log); + } + + @Override + public void log (int level, String category, String message, Throwable ex) { + final String logString = "[KRYO " + category + "] " + message; + switch (level) { + case Log.LEVEL_ERROR: + log.error(logString, ex); + break; + case Log.LEVEL_WARN: + log.warn(logString, ex); + break; + case Log.LEVEL_INFO: + log.info(logString, ex); + break; + case Log.LEVEL_DEBUG: + log.debug(logString, ex); + break; + case Log.LEVEL_TRACE: + log.trace(logString, ex); + break; + } + } +} From ebcca875ec82fc4ede0f8fed495ba48b23c73960 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 6 Mar 2018 11:30:54 +0100 Subject: [PATCH 081/268] [FLINK-8878] [tests] Add BlockerSync utility This helps to synchronize two threads of which one is expected to block while holding a resource. --- .../flink/core/testutils/BlockerSync.java | 107 ++++++++++++++++++ 1 file changed, 107 insertions(+) create mode 100644 flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/BlockerSync.java diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/BlockerSync.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/BlockerSync.java new file mode 100644 index 0000000000000..fb854f87d54bb --- /dev/null +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/core/testutils/BlockerSync.java @@ -0,0 +1,107 @@ +/* + * 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.core.testutils; + +/** + * A utility to help synchronize two threads in cases where one of them is supposed to reach + * a blocking state before the other may continue. + * + *

    Use as follows: + *

    + * {@code
    + *
    + * final BlockerSync sync = new BlockerSync();
    + *
    + * // thread to be blocked
    + * Runnable toBeBlocked = () -> {
    + *     // do something, like acquire a shared resource
    + *     sync.blockNonInterruptible();
    + *     // release resource
    + * }
    + *
    + * new Thread(toBeBlocked).start();
    + * sync.awaitBlocker();
    + *
    + * // do stuff that requires the other thread to still hold the resource
    + * sync.releaseBlocker();
    + * }
    + * 
    + */ +public class BlockerSync { + + private final Object lock = new Object(); + + private boolean blockerReady; + + private boolean blockerReleased; + + /** + * Waits until the blocking thread has entered the method {@link #block()} + * or {@link #blockNonInterruptible()}. + */ + public void awaitBlocker() throws InterruptedException { + synchronized (lock) { + while (!blockerReady) { + lock.wait(); + } + } + } + + /** + * Blocks until {@link #releaseBlocker()} is called or this thread is interrupted. + * Notifies the awaiting thread that waits in the method {@link #awaitBlocker()}. + */ + public void block() throws InterruptedException { + synchronized (lock) { + blockerReady = true; + lock.notifyAll(); + + while (!blockerReleased) { + lock.wait(); + } + } + } + + /** + * Blocks until {@link #releaseBlocker()} is called. + * Notifies the awaiting thread that waits in the method {@link #awaitBlocker()}. + */ + public void blockNonInterruptible() { + synchronized (lock) { + blockerReady = true; + lock.notifyAll(); + + while (!blockerReleased) { + try { + lock.wait(); + } catch (InterruptedException ignored) {} + } + } + } + + /** + * Lets the blocked thread continue. + */ + public void releaseBlocker() { + synchronized (lock) { + blockerReleased = true; + lock.notifyAll(); + } + } +} From 8a77dbf16febea72d389b2dc497e63cb768a3d2d Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Sun, 4 Mar 2018 12:20:17 +0100 Subject: [PATCH 082/268] [FLINK-8878] [core] Add concurrency check Kryo Serializer on DEBUG level --- .../runtime/kryo/KryoSerializer.java | 191 ++++++++++++------ .../kryo/KryoSerializerDebugInitHelper.java | 47 +++++ ...ializerConcurrencyCheckInactiveITCase.java | 62 ++++++ .../kryo/KryoSerializerConcurrencyTest.java | 95 +++++++++ 4 files changed, 338 insertions(+), 57 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerDebugInitHelper.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerConcurrencyCheckInactiveITCase.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerConcurrencyTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java index 06ba906c241b0..7c97c5c44e18d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java @@ -74,6 +74,12 @@ public class KryoSerializer extends TypeSerializer { private static final Logger LOG = LoggerFactory.getLogger(KryoSerializer.class); + /** Flag whether to check for concurrent thread access. + * Because this flag is static final, a value of 'false' allows the JIT compiler to eliminate + * the guarded code sections. */ + private static final boolean CONCURRENT_ACCESS_CHECK = + LOG.isDebugEnabled() || KryoSerializerDebugInitHelper.setToDebug; + static { configureKryoLogging(); } @@ -112,6 +118,9 @@ public class KryoSerializer extends TypeSerializer { private LinkedHashMap, Class>> registeredTypesWithSerializerClasses; private LinkedHashSet> registeredTypes; + // for debugging purposes + private transient volatile Thread currentThread; + // ------------------------------------------------------------------------ public KryoSerializer(Class type, ExecutionConfig executionConfig){ @@ -174,26 +183,38 @@ public T copy(T from) { if (from == null) { return null; } - checkKryoInitialized(); - try { - return kryo.copy(from); + + if (CONCURRENT_ACCESS_CHECK) { + enterExclusiveThread(); } - catch(KryoException ke) { - // kryo was unable to copy it, so we do it through serialization: - ByteArrayOutputStream baout = new ByteArrayOutputStream(); - Output output = new Output(baout); - kryo.writeObject(output, from); + try { + checkKryoInitialized(); + try { + return kryo.copy(from); + } + catch (KryoException ke) { + // kryo was unable to copy it, so we do it through serialization: + ByteArrayOutputStream baout = new ByteArrayOutputStream(); + Output output = new Output(baout); + + kryo.writeObject(output, from); - output.close(); + output.close(); - ByteArrayInputStream bain = new ByteArrayInputStream(baout.toByteArray()); - Input input = new Input(bain); + ByteArrayInputStream bain = new ByteArrayInputStream(baout.toByteArray()); + Input input = new Input(bain); - return (T)kryo.readObject(input, from.getClass()); + return (T)kryo.readObject(input, from.getClass()); + } + } + finally { + if (CONCURRENT_ACCESS_CHECK) { + exitExclusiveThread(); + } } } - + @Override public T copy(T from, T reuse) { return copy(from); @@ -206,35 +227,47 @@ public int getLength() { @Override public void serialize(T record, DataOutputView target) throws IOException { - checkKryoInitialized(); - if (target != previousOut) { - DataOutputViewStream outputStream = new DataOutputViewStream(target); - output = new Output(outputStream); - previousOut = target; - } - - // Sanity check: Make sure that the output is cleared/has been flushed by the last call - // otherwise data might be written multiple times in case of a previous EOFException - if (output.position() != 0) { - throw new IllegalStateException("The Kryo Output still contains data from a previous " + - "serialize call. It has to be flushed or cleared at the end of the serialize call."); + if (CONCURRENT_ACCESS_CHECK) { + enterExclusiveThread(); } try { - kryo.writeClassAndObject(output, record); - output.flush(); - } - catch (KryoException ke) { - // make sure that the Kryo output buffer is cleared in case that we can recover from - // the exception (e.g. EOFException which denotes buffer full) - output.clear(); - - Throwable cause = ke.getCause(); - if (cause instanceof EOFException) { - throw (EOFException) cause; + checkKryoInitialized(); + + if (target != previousOut) { + DataOutputViewStream outputStream = new DataOutputViewStream(target); + output = new Output(outputStream); + previousOut = target; + } + + // Sanity check: Make sure that the output is cleared/has been flushed by the last call + // otherwise data might be written multiple times in case of a previous EOFException + if (output.position() != 0) { + throw new IllegalStateException("The Kryo Output still contains data from a previous " + + "serialize call. It has to be flushed or cleared at the end of the serialize call."); + } + + try { + kryo.writeClassAndObject(output, record); + output.flush(); + } + catch (KryoException ke) { + // make sure that the Kryo output buffer is cleared in case that we can recover from + // the exception (e.g. EOFException which denotes buffer full) + output.clear(); + + Throwable cause = ke.getCause(); + if (cause instanceof EOFException) { + throw (EOFException) cause; + } + else { + throw ke; + } } - else { - throw ke; + } + finally { + if (CONCURRENT_ACCESS_CHECK) { + exitExclusiveThread(); } } } @@ -242,26 +275,38 @@ public void serialize(T record, DataOutputView target) throws IOException { @SuppressWarnings("unchecked") @Override public T deserialize(DataInputView source) throws IOException { - checkKryoInitialized(); - if (source != previousIn) { - DataInputViewStream inputStream = new DataInputViewStream(source); - input = new NoFetchingInput(inputStream); - previousIn = source; + if (CONCURRENT_ACCESS_CHECK) { + enterExclusiveThread(); } try { - return (T) kryo.readClassAndObject(input); - } catch (KryoException ke) { - Throwable cause = ke.getCause(); - - if (cause instanceof EOFException) { - throw (EOFException) cause; - } else { - throw ke; + checkKryoInitialized(); + + if (source != previousIn) { + DataInputViewStream inputStream = new DataInputViewStream(source); + input = new NoFetchingInput(inputStream); + previousIn = source; + } + + try { + return (T) kryo.readClassAndObject(input); + } catch (KryoException ke) { + Throwable cause = ke.getCause(); + + if (cause instanceof EOFException) { + throw (EOFException) cause; + } else { + throw ke; + } + } + } + finally { + if (CONCURRENT_ACCESS_CHECK) { + exitExclusiveThread(); } } } - + @Override public T deserialize(T reuse, DataInputView source) throws IOException { return deserialize(source); @@ -269,13 +314,24 @@ public T deserialize(T reuse, DataInputView source) throws IOException { @Override public void copy(DataInputView source, DataOutputView target) throws IOException { - checkKryoInitialized(); - if(this.copyInstance == null){ - this.copyInstance = createInstance(); + if (CONCURRENT_ACCESS_CHECK) { + enterExclusiveThread(); } - T tmp = deserialize(copyInstance, source); - serialize(tmp, target); + try { + checkKryoInitialized(); + if (this.copyInstance == null){ + this.copyInstance = createInstance(); + } + + T tmp = deserialize(copyInstance, source); + serialize(tmp, target); + } + finally { + if (CONCURRENT_ACCESS_CHECK) { + exitExclusiveThread(); + } + } } // -------------------------------------------------------------------------------------------- @@ -516,6 +572,27 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE // For testing // -------------------------------------------------------------------------------------------- + private void enterExclusiveThread() { + // we use simple get, check, set here, rather than CAS + // we don't need lock-style correctness, this is only a sanity-check and we thus + // favor speed at the cost of some false negatives in this check + Thread previous = currentThread; + Thread thisThread = Thread.currentThread(); + + if (previous == null) { + currentThread = thisThread; + } + else if (previous != thisThread) { + throw new IllegalStateException( + "Concurrent access to KryoSerializer. Thread 1: " + thisThread.getName() + + " , Thread 2: " + previous.getName()); + } + } + + private void exitExclusiveThread() { + currentThread = null; + } + @VisibleForTesting public Kryo getKryo() { checkKryoInitialized(); diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerDebugInitHelper.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerDebugInitHelper.java new file mode 100644 index 0000000000000..ac918d645f84a --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerDebugInitHelper.java @@ -0,0 +1,47 @@ +/* + * 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.api.java.typeutils.runtime.kryo; + +import org.apache.flink.annotation.Internal; + +/** + * Simple helper class to initialize the concurrency checks for tests. + * + *

    The flag is automatically set to true when assertions are activated (tests) + * and can be set to true manually in other tests as well; + */ +@Internal +class KryoSerializerDebugInitHelper { + + /** This captures the initial setting after initialization. It is used to + * validate in tests that we never change the default to true. */ + static final boolean INITIAL_SETTING; + + /** The flag that is used to initialize the KryoSerializer's concurrency check flag. */ + static boolean setToDebug = false; + + static { + // capture the default setting, for tests + INITIAL_SETTING = setToDebug; + + // if assertions are active, the check should be activated + //noinspection AssertWithSideEffects,ConstantConditions + assert setToDebug = true; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerConcurrencyCheckInactiveITCase.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerConcurrencyCheckInactiveITCase.java new file mode 100644 index 0000000000000..522bf9e9b303b --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerConcurrencyCheckInactiveITCase.java @@ -0,0 +1,62 @@ +/* + * 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.api.java.typeutils.runtime.kryo; + +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +/** + * A test that validates that the concurrency checks in the Kryo Serializer + * are not hard coded to active. + * + *

    The debug initialization in the KryoSerializer happens together with class + * initialization (that makes it peak efficient), which is why this test needs to + * run in a fresh JVM fork, and the JVM fork of this test should not be reused. + * + *

    Important: If you see this test fail and the initial settings are still + * correct, check the assumptions above (on fresh JVM fork). + */ +public class KryoSerializerConcurrencyCheckInactiveITCase { + + // this sets the debug initialization back to its default, even if + // by default tests modify it (implicitly via assertion loading) + static { + KryoSerializerDebugInitHelper.setToDebug = KryoSerializerDebugInitHelper.INITIAL_SETTING; + } + + /** + * This test checks that concurrent access is not detected by default, meaning that + * the thread concurrency checks are off by default. + */ + @Test + public void testWithNoConcurrencyCheck() throws Exception { + boolean assertionError; + try { + new KryoSerializerConcurrencyTest().testConcurrentUseOfSerializer(); + assertionError = false; + } + catch (AssertionError e) { + assertionError = true; + } + + assertTrue("testConcurrentUseOfSerializer() should have failed if " + + "concurrency checks are off by default", assertionError); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerConcurrencyTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerConcurrencyTest.java new file mode 100644 index 0000000000000..ca81fd4e6fa71 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerConcurrencyTest.java @@ -0,0 +1,95 @@ +/* + * 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.api.java.typeutils.runtime.kryo; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.testutils.BlockerSync; +import org.apache.flink.core.testutils.CheckedThread; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.fail; + +/** + * This tests that the {@link KryoSerializer} properly fails when accessed by two threads + * concurrently. + * + *

    Important: This test only works if assertions are activated (-ea) on the JVM + * when running tests. + */ +public class KryoSerializerConcurrencyTest { + + @Test + public void testConcurrentUseOfSerializer() throws Exception { + final KryoSerializer serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); + + final BlockerSync sync = new BlockerSync(); + + final DataOutputView regularOut = new DataOutputSerializer(32); + final DataOutputView lockingOut = new LockingView(sync); + + // this thread serializes and gets stuck there + final CheckedThread thread = new CheckedThread("serializer") { + @Override + public void go() throws Exception { + serializer.serialize("a value", lockingOut); + } + }; + + thread.start(); + sync.awaitBlocker(); + + // this should fail with an exception + try { + serializer.serialize("value", regularOut); + fail("should have failed with an exception"); + } + catch (IllegalStateException e) { + // expected + } + finally { + // release the thread that serializes + sync.releaseBlocker(); + } + + // this propagates exceptions from the spawned thread + thread.sync(); + } + + // ------------------------------------------------------------------------ + + private static class LockingView extends DataOutputSerializer { + + private final BlockerSync blocker; + + LockingView(BlockerSync blocker) { + super(32); + this.blocker = blocker; + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + blocker.blockNonInterruptible(); + } + } +} From 6ec1b784e5fea4d9d5208d44caf6fefde14f4aa8 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 6 Mar 2018 11:21:08 +0100 Subject: [PATCH 083/268] [FLINK-8879] [avro] Add concurrency check Avro Serializer on DEBUG level. --- .../avro/typeutils/AvroSerializer.java | 114 ++++++++++++++++-- .../AvroSerializerDebugInitHelper.java | 47 ++++++++ ...ializerConcurrencyCheckInactiveITCase.java | 62 ++++++++++ .../AvroSerializerConcurrencyTest.java | 94 +++++++++++++++ .../AvroSerializerSerializabilityTest.java | 70 +++++++++++ .../flink-1.4-serializer-java-serialized | Bin 0 -> 202 bytes 6 files changed, 374 insertions(+), 13 deletions(-) create mode 100644 flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializerDebugInitHelper.java create mode 100644 flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerConcurrencyCheckInactiveITCase.java create mode 100644 flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerConcurrencyTest.java create mode 100644 flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerSerializabilityTest.java create mode 100644 flink-formats/flink-avro/src/test/resources/flink-1.4-serializer-java-serialized diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java index bc3369fcf593a..75f298893e0e3 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java @@ -39,6 +39,8 @@ import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificDatumWriter; import org.apache.avro.specific.SpecificRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; @@ -52,12 +54,24 @@ * (ReflectDatumReader / -Writer). The serializer instantiates them depending on * the class of the type it should serialize. * + *

    Important: This serializer is NOT THREAD SAFE, because it reuses the data encoders + * and decoders which have buffers that would be shared between the threads if used concurrently + * * @param The type to be serialized. */ public class AvroSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; + /** Logger instance. */ + private static final Logger LOG = LoggerFactory.getLogger(AvroSerializer.class); + + /** Flag whether to check for concurrent thread access. + * Because this flag is static final, a value of 'false' allows the JIT compiler to eliminate + * the guarded code sections. */ + private static final boolean CONCURRENT_ACCESS_CHECK = + LOG.isDebugEnabled() || AvroSerializerDebugInitHelper.setToDebug; + // -------- configuration fields, serializable ----------- /** The class of the type that is serialized by this serializer. */ @@ -78,6 +92,9 @@ public class AvroSerializer extends TypeSerializer { /** The serializer configuration snapshot, cached for efficiency. */ private transient AvroSchemaSerializerConfigSnapshot configSnapshot; + /** The currently accessing thread, set and checked on debug level only. */ + private transient volatile Thread currentThread; + // ------------------------------------------------------------------------ /** @@ -127,23 +144,56 @@ public T createInstance() { @Override public void serialize(T value, DataOutputView target) throws IOException { - checkAvroInitialized(); - this.encoder.setOut(target); - this.writer.write(value, this.encoder); + if (CONCURRENT_ACCESS_CHECK) { + enterExclusiveThread(); + } + + try { + checkAvroInitialized(); + this.encoder.setOut(target); + this.writer.write(value, this.encoder); + } + finally { + if (CONCURRENT_ACCESS_CHECK) { + exitExclusiveThread(); + } + } } @Override public T deserialize(DataInputView source) throws IOException { - checkAvroInitialized(); - this.decoder.setIn(source); - return this.reader.read(null, this.decoder); + if (CONCURRENT_ACCESS_CHECK) { + enterExclusiveThread(); + } + + try { + checkAvroInitialized(); + this.decoder.setIn(source); + return this.reader.read(null, this.decoder); + } + finally { + if (CONCURRENT_ACCESS_CHECK) { + exitExclusiveThread(); + } + } } @Override public T deserialize(T reuse, DataInputView source) throws IOException { - checkAvroInitialized(); - this.decoder.setIn(source); - return this.reader.read(reuse, this.decoder); + if (CONCURRENT_ACCESS_CHECK) { + enterExclusiveThread(); + } + + try { + checkAvroInitialized(); + this.decoder.setIn(source); + return this.reader.read(reuse, this.decoder); + } + finally { + if (CONCURRENT_ACCESS_CHECK) { + exitExclusiveThread(); + } + } } // ------------------------------------------------------------------------ @@ -152,8 +202,19 @@ public T deserialize(T reuse, DataInputView source) throws IOException { @Override public T copy(T from) { - checkAvroInitialized(); - return avroData.deepCopy(schema, from); + if (CONCURRENT_ACCESS_CHECK) { + enterExclusiveThread(); + } + + try { + checkAvroInitialized(); + return avroData.deepCopy(schema, from); + } + finally { + if (CONCURRENT_ACCESS_CHECK) { + exitExclusiveThread(); + } + } } @Override @@ -163,8 +224,10 @@ public T copy(T from, T reuse) { @Override public void copy(DataInputView source, DataOutputView target) throws IOException { - T value = deserialize(source); - serialize(value, target); + // we do not have concurrency checks here, because serialize() and + // deserialize() do the checks and the current concurrency check mechanism + // does provide additional safety in cases of re-entrant calls + serialize(deserialize(source), target); } // ------------------------------------------------------------------------ @@ -277,6 +340,31 @@ private void initializeAvro() { this.decoder = new DataInputDecoder(); } + // -------------------------------------------------------------------------------------------- + // Concurrency checks + // -------------------------------------------------------------------------------------------- + + private void enterExclusiveThread() { + // we use simple get, check, set here, rather than CAS + // we don't need lock-style correctness, this is only a sanity-check and we thus + // favor speed at the cost of some false negatives in this check + Thread previous = currentThread; + Thread thisThread = Thread.currentThread(); + + if (previous == null) { + currentThread = thisThread; + } + else if (previous != thisThread) { + throw new IllegalStateException( + "Concurrent access to KryoSerializer. Thread 1: " + thisThread.getName() + + " , Thread 2: " + previous.getName()); + } + } + + private void exitExclusiveThread() { + currentThread = null; + } + // ------------------------------------------------------------------------ // Serializer Snapshots // ------------------------------------------------------------------------ diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializerDebugInitHelper.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializerDebugInitHelper.java new file mode 100644 index 0000000000000..c65709278e61b --- /dev/null +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializerDebugInitHelper.java @@ -0,0 +1,47 @@ +/* + * 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.formats.avro.typeutils; + +import org.apache.flink.annotation.Internal; + +/** + * Simple helper class to initialize the concurrency checks for tests. + * + *

    The flag is automatically set to true when assertions are activated (tests) + * and can be set to true manually in other tests as well; + */ +@Internal +class AvroSerializerDebugInitHelper { + + /** This captures the initial setting after initialization. It is used to + * validate in tests that we never change the default to true. */ + static final boolean INITIAL_SETTING; + + /** The flag that is used to initialize the KryoSerializer's concurrency check flag. */ + static boolean setToDebug = false; + + static { + // capture the default setting, for tests + INITIAL_SETTING = setToDebug; + + // if assertions are active, the check should be activated + //noinspection AssertWithSideEffects,ConstantConditions + assert setToDebug = true; + } +} diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerConcurrencyCheckInactiveITCase.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerConcurrencyCheckInactiveITCase.java new file mode 100644 index 0000000000000..9b98e44447ccf --- /dev/null +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerConcurrencyCheckInactiveITCase.java @@ -0,0 +1,62 @@ +/* + * 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.formats.avro.typeutils; + +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +/** + * A test that validates that the concurrency checks in the Avro Serializer + * are not hard coded to active. + * + *

    The debug initialization in the AvroSerializer happens together with class + * initialization (that makes it peak efficient), which is why this test needs to + * run in a fresh JVM fork, and the JVM fork of this test should not be reused. + * + *

    Important: If you see this test fail and the initial settings are still + * correct, check the assumptions above (on fresh JVM fork). + */ +public class AvroSerializerConcurrencyCheckInactiveITCase { + + // this sets the debug initialization back to its default, even if + // by default tests modify it (implicitly via assertion loading) + static { + AvroSerializerDebugInitHelper.setToDebug = AvroSerializerDebugInitHelper.INITIAL_SETTING; + } + + /** + * This test checks that concurrent access is not detected by default, meaning that + * the thread concurrency checks are off by default. + */ + @Test + public void testWithNoConcurrencyCheck() throws Exception { + boolean assertionError; + try { + new AvroSerializerConcurrencyTest().testConcurrentUseOfSerializer(); + assertionError = false; + } + catch (AssertionError e) { + assertionError = true; + } + + assertTrue("testConcurrentUseOfSerializer() should have failed if " + + "concurrency checks are off by default", assertionError); + } +} diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerConcurrencyTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerConcurrencyTest.java new file mode 100644 index 0000000000000..aaa9b4b08b70c --- /dev/null +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerConcurrencyTest.java @@ -0,0 +1,94 @@ +/* + * 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.formats.avro.typeutils; + +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.testutils.BlockerSync; +import org.apache.flink.core.testutils.CheckedThread; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.fail; + +/** + * This tests that the {@link AvroSerializer} properly fails when accessed by two threads + * concurrently. + * + *

    Important: This test only works if assertions are activated (-ea) on the JVM + * when running tests. + */ +public class AvroSerializerConcurrencyTest { + + @Test + public void testConcurrentUseOfSerializer() throws Exception { + final AvroSerializer serializer = new AvroSerializer<>(String.class); + + final BlockerSync sync = new BlockerSync(); + + final DataOutputView regularOut = new DataOutputSerializer(32); + final DataOutputView lockingOut = new LockingView(sync); + + // this thread serializes and gets stuck there + final CheckedThread thread = new CheckedThread("serializer") { + @Override + public void go() throws Exception { + serializer.serialize("a value", lockingOut); + } + }; + + thread.start(); + sync.awaitBlocker(); + + // this should fail with an exception + try { + serializer.serialize("value", regularOut); + fail("should have failed with an exception"); + } + catch (IllegalStateException e) { + // expected + } + finally { + // release the thread that serializes + sync.releaseBlocker(); + } + + // this propagates exceptions from the spawned thread + thread.sync(); + } + + // ------------------------------------------------------------------------ + + private static class LockingView extends DataOutputSerializer { + + private final BlockerSync blocker; + + LockingView(BlockerSync blocker) { + super(32); + this.blocker = blocker; + } + + @Override + public void writeInt(int v) throws IOException { + blocker.blockNonInterruptible(); + } + } +} diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerSerializabilityTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerSerializabilityTest.java new file mode 100644 index 0000000000000..c15aa7c01414b --- /dev/null +++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/AvroSerializerSerializabilityTest.java @@ -0,0 +1,70 @@ +/* + * 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.formats.avro.typeutils; + +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +import static org.junit.Assert.assertEquals; + +/** + * Test that validates that the serialized form of the AvroSerializer is the same as in + * previous Flink versions. + * + *

    While that is not strictly necessary for FLink to work, it increases user experience + * in job upgrade situations. + */ +public class AvroSerializerSerializabilityTest { + + private static final String RESOURCE_NAME = "flink-1.4-serializer-java-serialized"; + + @Test + public void testDeserializeSerializer() throws Exception { + final AvroSerializer currentSerializer = new AvroSerializer<>(String.class); + + try (ObjectInputStream in = new ObjectInputStream( + getClass().getClassLoader().getResourceAsStream(RESOURCE_NAME))) { + + @SuppressWarnings("unchecked") + AvroSerializer deserialized = (AvroSerializer) in.readObject(); + + assertEquals(currentSerializer, deserialized); + } + } + + // ------------------------------------------------------------------------ + // To create a serialized serializer file + // ------------------------------------------------------------------------ + + public static void main(String[] args) throws Exception { + final AvroSerializer serializer = new AvroSerializer<>(String.class); + + final File file = new File("flink-formats/flink-avro/src/test/resources/" + RESOURCE_NAME).getAbsoluteFile(); + + try (FileOutputStream fos = new FileOutputStream(file); + ObjectOutputStream out = new ObjectOutputStream(fos)) { + + out.writeObject(serializer); + } + } +} diff --git a/flink-formats/flink-avro/src/test/resources/flink-1.4-serializer-java-serialized b/flink-formats/flink-avro/src/test/resources/flink-1.4-serializer-java-serialized new file mode 100644 index 0000000000000000000000000000000000000000..63fef0a3dabfc6a49a1441de4e6c72352d981073 GIT binary patch literal 202 zcmZw9F$%&k6vpw_qPt$?brBIdY1eM`0*{i$XqtqV7-~=B;OOSL1dm{Yi=f~3!Jps# z33Jb|wk(kxwFiNnR_27A<(`5^ifl1loHz%iJ#MGl2v(%kC>TD=a>#0!r7FO(K2oGY xlaWQG$@_9(SpE3q6t=eSt@+aTlkiW6!9|8ef-%9^1Xe~~U$@OzK6W`w!5i#!Li+#! literal 0 HcmV?d00001 From 27ccf196913f0f9ba20d54e370cfbb08ec1cbb7e Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Thu, 8 Mar 2018 11:22:19 +0100 Subject: [PATCH 084/268] [hotfix] [build] Change REST port to 8081 for end-to-end testing scripts Now that the FLIP-6 code uses 8081, we need to probe that port to check Flink's status in the end-to-end tests. --- flink-end-to-end-tests/test-scripts/common.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index e6d21a26b0907..ef4856f561b54 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -45,7 +45,7 @@ function start_cluster { # wait at most 10 seconds until the dispatcher is up for i in {1..10}; do # without the || true this would exit our script if the JobManager is not yet up - QUERY_RESULT=$(curl "http://localhost:9065/taskmanagers" 2> /dev/null || true) + QUERY_RESULT=$(curl "http://localhost:8081/taskmanagers" 2> /dev/null || true) if [[ "$QUERY_RESULT" == "" ]]; then echo "Dispatcher/TaskManagers are not yet up" From 5cf53bd2e430ccfcb9fb0e7c0e31008acc6a22cb Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 5 Mar 2018 13:46:41 +0100 Subject: [PATCH 085/268] [FLINK-8839] [sql-client] Fix table source factory discovery This closes #5640. --- flink-libraries/flink-sql-client/pom.xml | 23 +++ .../flink/table/client/gateway/Executor.java | 18 +- .../table/client/gateway/SessionContext.java | 20 ++ .../gateway/local/ExecutionContext.java | 166 +++++++++++++++ .../client/gateway/local/LocalExecutor.java | 189 ++++++------------ .../assembly/test-table-source-factory.xml | 47 +++++ .../client/gateway/local/DependencyTest.java | 72 +++++++ .../gateway/local/LocalExecutorITCase.java | 59 +++--- .../gateway/utils/EnvironmentFileUtil.java | 56 ++++++ .../gateway/utils/TestTableSourceFactory.java | 112 +++++++++++ .../test/resources/test-factory-services-file | 20 ++ .../resources/test-sql-client-defaults.yaml | 1 + .../resources/test-sql-client-factory.yaml | 45 +++++ .../flink/table/descriptors/Rowtime.scala | 2 +- .../sources/TableSourceFactoryService.scala | 27 ++- .../flink/table/descriptors/RowtimeTest.scala | 2 +- 16 files changed, 683 insertions(+), 176 deletions(-) create mode 100644 flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java create mode 100644 flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml create mode 100644 flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java create mode 100644 flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/EnvironmentFileUtil.java create mode 100644 flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java create mode 100644 flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file create mode 100644 flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml diff --git a/flink-libraries/flink-sql-client/pom.xml b/flink-libraries/flink-sql-client/pom.xml index 743f58308b7ce..71c0812c21226 100644 --- a/flink-libraries/flink-sql-client/pom.xml +++ b/flink-libraries/flink-sql-client/pom.xml @@ -133,6 +133,7 @@ under the License. + org.apache.maven.plugins maven-shade-plugin @@ -167,6 +168,28 @@ under the License. + + + + maven-assembly-plugin + 2.4 + + + create-table-source-factory-jar + process-test-classes + + single + + + table-source-factory + false + + src/test/assembly/test-table-source-factory.xml + + + + + diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java index 512c194393713..4a41222700e26 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java @@ -38,38 +38,38 @@ public interface Executor { /** * Lists all session properties that are defined by the executor and the session. */ - Map getSessionProperties(SessionContext context) throws SqlExecutionException; + Map getSessionProperties(SessionContext session) throws SqlExecutionException; /** * Lists all tables known to the executor. */ - List listTables(SessionContext context) throws SqlExecutionException; + List listTables(SessionContext session) throws SqlExecutionException; /** * Returns the schema of a table. Throws an exception if the table could not be found. */ - TableSchema getTableSchema(SessionContext context, String name) throws SqlExecutionException; + TableSchema getTableSchema(SessionContext session, String name) throws SqlExecutionException; /** * Returns a string-based explanation about AST and execution plan of the given statement. */ - String explainStatement(SessionContext context, String statement) throws SqlExecutionException; + String explainStatement(SessionContext session, String statement) throws SqlExecutionException; /** * Submits a Flink job (detached) and returns the result descriptor. */ - ResultDescriptor executeQuery(SessionContext context, String query) throws SqlExecutionException; + ResultDescriptor executeQuery(SessionContext session, String query) throws SqlExecutionException; /** * Asks for the next changelog results (non-blocking). */ - TypedResult>> retrieveResultChanges(SessionContext context, String resultId) throws SqlExecutionException; + TypedResult>> retrieveResultChanges(SessionContext session, String resultId) throws SqlExecutionException; /** * Creates an immutable result snapshot of the running Flink job. Throws an exception if no Flink job can be found. * Returns the number of pages. */ - TypedResult snapshotResult(SessionContext context, String resultId, int pageSize) throws SqlExecutionException; + TypedResult snapshotResult(SessionContext session, String resultId, int pageSize) throws SqlExecutionException; /** * Returns the rows that are part of the current page or throws an exception if the snapshot has been expired. @@ -79,10 +79,10 @@ public interface Executor { /** * Cancels a table program and stops the result retrieval. */ - void cancelQuery(SessionContext context, String resultId) throws SqlExecutionException; + void cancelQuery(SessionContext session, String resultId) throws SqlExecutionException; /** * Stops the executor. */ - void stop(SessionContext context); + void stop(SessionContext session); } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/SessionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/SessionContext.java index 1058eb6a3f60f..0b6ee2c8ef562 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/SessionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/SessionContext.java @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.Objects; /** * Context describing a session. @@ -54,4 +55,23 @@ public Environment getEnvironment() { // enrich with session properties return Environment.enrich(defaultEnvironment, sessionProperties); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof SessionContext)) { + return false; + } + SessionContext context = (SessionContext) o; + return Objects.equals(name, context.name) && + Objects.equals(defaultEnvironment, context.defaultEnvironment) && + Objects.equals(sessionProperties, context.sessionProperties); + } + + @Override + public int hashCode() { + return Objects.hash(name, defaultEnvironment, sessionProperties); + } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java new file mode 100644 index 0000000000000..15a3c129bb801 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -0,0 +1,166 @@ +/* + * 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.table.client.gateway.local; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.Plan; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.optimizer.DataStatistics; +import org.apache.flink.optimizer.Optimizer; +import org.apache.flink.optimizer.costs.DefaultCostEstimator; +import org.apache.flink.optimizer.plan.FlinkPlan; +import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.table.api.BatchQueryConfig; +import org.apache.flink.table.api.QueryConfig; +import org.apache.flink.table.api.StreamQueryConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.client.config.Environment; +import org.apache.flink.table.client.gateway.SessionContext; +import org.apache.flink.table.sources.TableSource; +import org.apache.flink.table.sources.TableSourceFactoryService; + +import java.net.URL; +import java.util.List; + +/** + * Context for executing table programs. It contains configured environments and environment + * specific logic such as plan translation. + */ +public class ExecutionContext { + + private final SessionContext sessionContext; + private final Environment mergedEnv; + private final ExecutionEnvironment execEnv; + private final StreamExecutionEnvironment streamExecEnv; + private final TableEnvironment tableEnv; + private final ClassLoader classLoader; + private final QueryConfig queryConfig; + + public ExecutionContext(Environment defaultEnvironment, SessionContext sessionContext, List dependencies) { + this.sessionContext = sessionContext; + this.mergedEnv = Environment.merge(defaultEnvironment, sessionContext.getEnvironment()); + + // create environments + if (mergedEnv.getExecution().isStreamingExecution()) { + streamExecEnv = createStreamExecutionEnvironment(); + execEnv = null; + tableEnv = TableEnvironment.getTableEnvironment(streamExecEnv); + } else { + streamExecEnv = null; + execEnv = createExecutionEnvironment(); + tableEnv = TableEnvironment.getTableEnvironment(execEnv); + } + + // create class loader + classLoader = FlinkUserCodeClassLoaders.parentFirst( + dependencies.toArray(new URL[dependencies.size()]), + this.getClass().getClassLoader()); + + // create table sources + mergedEnv.getSources().forEach((name, source) -> { + TableSource tableSource = TableSourceFactoryService.findAndCreateTableSource(source, classLoader); + tableEnv.registerTableSource(name, tableSource); + }); + + // create query config + queryConfig = createQueryConfig(); + } + + public SessionContext getSessionContext() { + return sessionContext; + } + + public ExecutionEnvironment getExecutionEnvironment() { + return execEnv; + } + + public StreamExecutionEnvironment getStreamExecutionEnvironment() { + return streamExecEnv; + } + + public TableEnvironment getTableEnvironment() { + return tableEnv; + } + + public ClassLoader getClassLoader() { + return classLoader; + } + + public Environment getMergedEnvironment() { + return mergedEnv; + } + + public QueryConfig getQueryConfig() { + return queryConfig; + } + + public ExecutionConfig getExecutionConfig() { + if (streamExecEnv != null) { + return streamExecEnv.getConfig(); + } else { + return execEnv.getConfig(); + } + } + + public FlinkPlan createPlan(String name, Configuration flinkConfig) { + if (streamExecEnv != null) { + final StreamGraph graph = streamExecEnv.getStreamGraph(); + graph.setJobName(name); + return graph; + } else { + final int parallelism = execEnv.getParallelism(); + final Plan unoptimizedPlan = execEnv.createProgramPlan(); + unoptimizedPlan.setJobName(name); + final Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig); + return ClusterClient.getOptimizedPlan(compiler, unoptimizedPlan, parallelism); + } + } + + // -------------------------------------------------------------------------------------------- + + private ExecutionEnvironment createExecutionEnvironment() { + final ExecutionEnvironment execEnv = ExecutionEnvironment.getExecutionEnvironment(); + execEnv.setParallelism(mergedEnv.getExecution().getParallelism()); + return execEnv; + } + + private StreamExecutionEnvironment createStreamExecutionEnvironment() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(mergedEnv.getExecution().getParallelism()); + env.setMaxParallelism(mergedEnv.getExecution().getMaxParallelism()); + return env; + } + + private QueryConfig createQueryConfig() { + if (streamExecEnv != null) { + final StreamQueryConfig config = new StreamQueryConfig(); + final long minRetention = mergedEnv.getExecution().getMinStateRetention(); + final long maxRetention = mergedEnv.getExecution().getMaxStateRetention(); + config.withIdleStateRetentionTime(Time.milliseconds(minRetention), Time.milliseconds(maxRetention)); + return config; + } else { + return new BatchQueryConfig(); + } + } +} diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index 8c40885d36a72..35d7da9bbfd28 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -18,11 +18,7 @@ package org.apache.flink.table.client.gateway.local; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.client.deployment.ClusterDescriptor; @@ -36,34 +32,21 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.core.fs.Path; -import org.apache.flink.optimizer.DataStatistics; -import org.apache.flink.optimizer.Optimizer; -import org.apache.flink.optimizer.costs.DefaultCostEstimator; import org.apache.flink.optimizer.plan.FlinkPlan; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.table.api.BatchQueryConfig; -import org.apache.flink.table.api.QueryConfig; -import org.apache.flink.table.api.StreamQueryConfig; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.java.BatchTableEnvironment; -import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.client.SqlClientException; import org.apache.flink.table.client.config.Deployment; import org.apache.flink.table.client.config.Environment; -import org.apache.flink.table.client.config.Execution; import org.apache.flink.table.client.gateway.Executor; import org.apache.flink.table.client.gateway.ResultDescriptor; import org.apache.flink.table.client.gateway.SessionContext; import org.apache.flink.table.client.gateway.SqlExecutionException; import org.apache.flink.table.client.gateway.TypedResult; import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.sources.TableSource; -import org.apache.flink.table.sources.TableSourceFactoryService; import org.apache.flink.types.Row; import org.apache.flink.util.StringUtils; @@ -86,11 +69,17 @@ public class LocalExecutor implements Executor { private static final String DEFAULT_ENV_FILE = "sql-client-defaults.yaml"; - private final Environment environment; + private final Environment defaultEnvironment; private final List dependencies; private final Configuration flinkConfig; private final ResultStore resultStore; + /** + * Cached execution context for unmodified sessions. Do not access this variable directly + * but through {@link LocalExecutor#getOrCreateExecutionContext}. + */ + private ExecutionContext executionContext; + /** * Creates a local executor for submitting table programs and retrieving results. */ @@ -129,12 +118,12 @@ public LocalExecutor(URL defaultEnv, List jars, List libraries) { if (defaultEnv != null) { System.out.println("Reading default environment from: " + defaultEnv); try { - environment = Environment.parse(defaultEnv); + defaultEnvironment = Environment.parse(defaultEnv); } catch (IOException e) { throw new SqlClientException("Could not read default environment file at: " + defaultEnv, e); } } else { - environment = new Environment(); + defaultEnvironment = new Environment(); } // discover dependencies @@ -175,8 +164,11 @@ public LocalExecutor(URL defaultEnv, List jars, List libraries) { resultStore = new ResultStore(flinkConfig); } - public LocalExecutor(Environment environment, List dependencies, Configuration flinkConfig) { - this.environment = environment; + /** + * Constructor for testing purposes. + */ + public LocalExecutor(Environment defaultEnvironment, List dependencies, Configuration flinkConfig) { + this.defaultEnvironment = defaultEnvironment; this.dependencies = dependencies; this.flinkConfig = flinkConfig; @@ -190,8 +182,8 @@ public void start() { } @Override - public Map getSessionProperties(SessionContext context) throws SqlExecutionException { - final Environment env = createEnvironment(context); + public Map getSessionProperties(SessionContext session) throws SqlExecutionException { + final Environment env = getOrCreateExecutionContext(session).getMergedEnvironment(); final Map properties = new HashMap<>(); properties.putAll(env.getExecution().toProperties()); properties.putAll(env.getDeployment().toProperties()); @@ -199,16 +191,14 @@ public Map getSessionProperties(SessionContext context) throws S } @Override - public List listTables(SessionContext context) throws SqlExecutionException { - final Environment env = createEnvironment(context); - final TableEnvironment tableEnv = createTableEnvironment(env); + public List listTables(SessionContext session) throws SqlExecutionException { + final TableEnvironment tableEnv = getOrCreateExecutionContext(session).getTableEnvironment(); return Arrays.asList(tableEnv.listTables()); } @Override - public TableSchema getTableSchema(SessionContext context, String name) throws SqlExecutionException { - final Environment env = createEnvironment(context); - final TableEnvironment tableEnv = createTableEnvironment(env); + public TableSchema getTableSchema(SessionContext session, String name) throws SqlExecutionException { + final TableEnvironment tableEnv = getOrCreateExecutionContext(session).getTableEnvironment(); try { return tableEnv.scan(name).getSchema(); } catch (Throwable t) { @@ -218,13 +208,13 @@ public TableSchema getTableSchema(SessionContext context, String name) throws Sq } @Override - public String explainStatement(SessionContext context, String statement) throws SqlExecutionException { - final Environment env = createEnvironment(context); + public String explainStatement(SessionContext session, String statement) throws SqlExecutionException { + final ExecutionContext context = getOrCreateExecutionContext(session); // translate try { - final Tuple2 table = createTable(env, statement); - return table.f0.explain(table.f1); + final Table table = createTable(context, statement); + return context.getTableEnvironment().explain(table); } catch (Throwable t) { // catch everything such that the query does not crash the executor throw new SqlExecutionException("Invalid SQL statement.", t); @@ -232,29 +222,26 @@ public String explainStatement(SessionContext context, String statement) throws } @Override - public ResultDescriptor executeQuery(SessionContext context, String query) throws SqlExecutionException { - final Environment env = createEnvironment(context); + public ResultDescriptor executeQuery(SessionContext session, String query) throws SqlExecutionException { + final ExecutionContext context = getOrCreateExecutionContext(session); + final Environment mergedEnv = context.getMergedEnvironment(); // create table here to fail quickly for wrong queries - final Tuple2 table = createTable(env, query); + final Table table = createTable(context, query); // deployment - final ClusterClient clusterClient = createDeployment(env.getDeployment()); + final ClusterClient clusterClient = createDeployment(mergedEnv.getDeployment()); // initialize result final DynamicResult result = resultStore.createResult( - env, - table.f1.getSchema(), - getExecutionConfig(table.f0)); + mergedEnv, + table.getSchema(), + context.getExecutionConfig()); // create job graph with jars final JobGraph jobGraph; try { - jobGraph = createJobGraph( - context.getName() + ": " + query, - env.getExecution(), - table.f0, - table.f1, + jobGraph = createJobGraph(context, context.getSessionContext().getName() + ": " + query, table, result.getTableSink(), clusterClient); } catch (Throwable t) { @@ -268,18 +255,12 @@ public ResultDescriptor executeQuery(SessionContext context, String query) throw final String resultId = jobGraph.getJobID().toString(); resultStore.storeResult(resultId, result); - // create class loader - final ClassLoader classLoader = JobWithJars.buildUserCodeClassLoader( - dependencies, - Collections.emptyList(), - this.getClass().getClassLoader()); - // create execution final Runnable program = () -> { // we need to submit the job attached for now // otherwise it is not possible to retrieve the reason why an execution failed try { - clusterClient.run(jobGraph, classLoader); + clusterClient.run(jobGraph, context.getClassLoader()); } catch (ProgramInvocationException e) { throw new SqlExecutionException("Could not execute table program.", e); } finally { @@ -294,11 +275,12 @@ public ResultDescriptor executeQuery(SessionContext context, String query) throw // start result retrieval result.startRetrieval(program); - return new ResultDescriptor(resultId, table.f1.getSchema(), result.isMaterialized()); + return new ResultDescriptor(resultId, table.getSchema(), result.isMaterialized()); } @Override - public TypedResult>> retrieveResultChanges(SessionContext context, String resultId) throws SqlExecutionException { + public TypedResult>> retrieveResultChanges(SessionContext session, + String resultId) throws SqlExecutionException { final DynamicResult result = resultStore.getResult(resultId); if (result == null) { throw new SqlExecutionException("Could not find a result with result identifier '" + resultId + "'."); @@ -310,7 +292,7 @@ public TypedResult>> retrieveResultChanges(SessionCont } @Override - public TypedResult snapshotResult(SessionContext context, String resultId, int pageSize) throws SqlExecutionException { + public TypedResult snapshotResult(SessionContext session, String resultId, int pageSize) throws SqlExecutionException { final DynamicResult result = resultStore.getResult(resultId); if (result == null) { throw new SqlExecutionException("Could not find a result with result identifier '" + resultId + "'."); @@ -334,7 +316,7 @@ public List retrieveResultPage(String resultId, int page) throws SqlExecuti } @Override - public void cancelQuery(SessionContext context, String resultId) throws SqlExecutionException { + public void cancelQuery(SessionContext session, String resultId) throws SqlExecutionException { final DynamicResult result = resultStore.getResult(resultId); if (result == null) { throw new SqlExecutionException("Could not find a result with result identifier '" + resultId + "'."); @@ -345,8 +327,8 @@ public void cancelQuery(SessionContext context, String resultId) throws SqlExecu resultStore.removeResult(resultId); // stop Flink job - final Environment env = createEnvironment(context); - final ClusterClient clusterClient = createDeployment(env.getDeployment()); + final Environment mergedEnv = getOrCreateExecutionContext(session).getMergedEnvironment(); + final ClusterClient clusterClient = createDeployment(mergedEnv.getDeployment()); try { clusterClient.cancel(new JobID(StringUtils.hexStringToByte(resultId))); } catch (Throwable t) { @@ -361,10 +343,10 @@ public void cancelQuery(SessionContext context, String resultId) throws SqlExecu } @Override - public void stop(SessionContext context) { + public void stop(SessionContext session) { resultStore.getResults().forEach((resultId) -> { try { - cancelQuery(context, resultId); + cancelQuery(session, resultId); } catch (Throwable t) { // ignore any throwable to keep the clean up running } @@ -373,45 +355,29 @@ public void stop(SessionContext context) { // -------------------------------------------------------------------------------------------- - private Tuple2 createTable(Environment env, String query) { - final TableEnvironment tableEnv = createTableEnvironment(env); - + private Table createTable(ExecutionContext context, String query) { // parse and validate query try { - return Tuple2.of(tableEnv, tableEnv.sqlQuery(query)); + return context.getTableEnvironment().sqlQuery(query); } catch (Throwable t) { // catch everything such that the query does not crash the executor throw new SqlExecutionException("Invalid SQL statement.", t); } } - private JobGraph createJobGraph(String name, Execution exec, TableEnvironment tableEnv, - Table table, TableSink sink, ClusterClient clusterClient) { - - final QueryConfig queryConfig = createQueryConfig(exec); + private JobGraph createJobGraph(ExecutionContext context, String name, Table table, + TableSink sink, ClusterClient clusterClient) { // translate try { - table.writeToSink(sink, queryConfig); + table.writeToSink(sink, context.getQueryConfig()); } catch (Throwable t) { // catch everything such that the query does not crash the executor throw new SqlExecutionException("Invalid SQL statement.", t); } // extract plan - final FlinkPlan plan; - if (exec.isStreamingExecution()) { - final StreamGraph graph = ((StreamTableEnvironment) tableEnv).execEnv().getStreamGraph(); - graph.setJobName(name); - plan = graph; - } else { - final int parallelism = exec.getParallelism(); - final Plan unoptimizedPlan = ((BatchTableEnvironment) tableEnv).execEnv().createProgramPlan(); - unoptimizedPlan.setJobName(name); - final Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), - clusterClient.getFlinkConfiguration()); - plan = ClusterClient.getOptimizedPlan(compiler, unoptimizedPlan, parallelism); - } + final FlinkPlan plan = context.createPlan(name, clusterClient.getFlinkConfiguration()); // create job graph return clusterClient.getJobGraph( @@ -421,15 +387,6 @@ private JobGraph createJobGraph(String name, Execution exec, TableEnvironment ta SavepointRestoreSettings.none()); } - @SuppressWarnings("unchecked") - private ExecutionConfig getExecutionConfig(TableEnvironment tableEnv) { - if (tableEnv instanceof StreamTableEnvironment) { - return ((StreamTableEnvironment) tableEnv).execEnv().getConfig(); - } else { - return ((BatchTableEnvironment) tableEnv).execEnv().getConfig(); - } - } - private ClusterClient createDeployment(Deployment deploy) { // change some configuration options for being more responsive @@ -457,44 +414,18 @@ private ClusterClient createStandaloneClusterClient(Configuration configurati } } - private Environment createEnvironment(SessionContext context) { - return Environment.merge(environment, context.getEnvironment()); - } - - private TableEnvironment createTableEnvironment(Environment env) { - try { - final TableEnvironment tableEnv; - if (env.getExecution().isStreamingExecution()) { - final StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment(); - execEnv.setParallelism(env.getExecution().getParallelism()); - execEnv.setMaxParallelism(env.getExecution().getMaxParallelism()); - tableEnv = StreamTableEnvironment.getTableEnvironment(execEnv); - } else { - final ExecutionEnvironment execEnv = ExecutionEnvironment.getExecutionEnvironment(); - execEnv.setParallelism(env.getExecution().getParallelism()); - tableEnv = BatchTableEnvironment.getTableEnvironment(execEnv); + /** + * Creates or reuses the execution context. + */ + private synchronized ExecutionContext getOrCreateExecutionContext(SessionContext session) throws SqlExecutionException { + if (executionContext == null || !executionContext.getSessionContext().equals(session)) { + try { + executionContext = new ExecutionContext(defaultEnvironment, session, dependencies); + } catch (Throwable t) { + // catch everything such that a configuration does not crash the executor + throw new SqlExecutionException("Could not create execution context.", t); } - - env.getSources().forEach((name, source) -> { - TableSource tableSource = TableSourceFactoryService.findAndCreateTableSource(source); - tableEnv.registerTableSource(name, tableSource); - }); - - return tableEnv; - } catch (Exception e) { - throw new SqlExecutionException("Could not create table environment.", e); - } - } - - private QueryConfig createQueryConfig(Execution exec) { - if (exec.isStreamingExecution()) { - final StreamQueryConfig config = new StreamQueryConfig(); - final long minRetention = exec.getMinStateRetention(); - final long maxRetention = exec.getMaxStateRetention(); - config.withIdleStateRetentionTime(Time.milliseconds(minRetention), Time.milliseconds(maxRetention)); - return config; - } else { - return new BatchQueryConfig(); } + return executionContext; } } diff --git a/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml b/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml new file mode 100644 index 0000000000000..fb9673c593ed3 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/assembly/test-table-source-factory.xml @@ -0,0 +1,47 @@ + + + + test-jar + + jar + + false + + + ${project.build.testOutputDirectory} + / + + + org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.class + org/apache/flink/table/client/gateway/utils/TestTableSourceFactory$*.class + + + + + + + src/test/resources/test-factory-services-file + META-INF/services + org.apache.flink.table.sources.TableSourceFactory + 0755 + + + diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java new file mode 100644 index 0000000000000..715d2db5c39c2 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java @@ -0,0 +1,72 @@ +/* + * 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.table.client.gateway.local; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.Types; +import org.apache.flink.table.client.config.Environment; +import org.apache.flink.table.client.gateway.SessionContext; +import org.apache.flink.table.client.gateway.utils.EnvironmentFileUtil; + +import org.junit.Test; + +import java.net.URL; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Dependency tests for {@link LocalExecutor}. Mainly for testing classloading of dependencies. + */ +public class DependencyTest { + + private static final String FACTORY_ENVIRONMENT_FILE = "test-sql-client-factory.yaml"; + private static final String TABLE_SOURCE_FACTORY_JAR_FILE = "table-source-factory-test-jar.jar"; + + @Test + public void testTableSourceFactoryDiscovery() throws Exception { + // create environment + final Map replaceVars = new HashMap<>(); + replaceVars.put("$VAR_0", "test-table-source-factory"); + replaceVars.put("$VAR_1", "test-property"); + replaceVars.put("$VAR_2", "test-value"); + final Environment env = EnvironmentFileUtil.parseModified(FACTORY_ENVIRONMENT_FILE, replaceVars); + + // create executor with dependencies + final URL dependency = Paths.get("target", TABLE_SOURCE_FACTORY_JAR_FILE).toUri().toURL(); + final LocalExecutor executor = new LocalExecutor( + env, + Collections.singletonList(dependency), + new Configuration()); + + final SessionContext session = new SessionContext("test-session", new Environment()); + + final TableSchema result = executor.getTableSchema(session, "TableNumber1"); + final TableSchema expected = TableSchema.builder() + .field("IntegerField1", Types.INT()) + .field("StringField1", Types.STRING()) + .build(); + + assertEquals(expected, result); + } +} diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java index f30cafe890ee9..a2ae28108bfb9 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java @@ -31,16 +31,15 @@ import org.apache.flink.table.client.gateway.ResultDescriptor; import org.apache.flink.table.client.gateway.SessionContext; import org.apache.flink.table.client.gateway.TypedResult; +import org.apache.flink.table.client.gateway.utils.EnvironmentFileUtil; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; -import org.apache.flink.util.FileUtils; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import java.io.File; import java.net.URL; import java.util.ArrayList; import java.util.Arrays; @@ -61,6 +60,8 @@ */ public class LocalExecutorITCase extends TestLogger { + private static final String DEFAULTS_ENVIRONMENT_FILE = "test-sql-client-defaults.yaml"; + private static StandaloneMiniCluster cluster; @BeforeClass @@ -80,9 +81,9 @@ public static void after() throws Exception { @Test public void testListTables() throws Exception { final Executor executor = createDefaultExecutor(); - final SessionContext context = new SessionContext("test-session", new Environment()); + final SessionContext session = new SessionContext("test-session", new Environment()); - final List actualTables = executor.listTables(context); + final List actualTables = executor.listTables(session); final List expectedTables = Arrays.asList("TableNumber1", "TableNumber2"); assertEquals(expectedTables, actualTables); @@ -91,12 +92,12 @@ public void testListTables() throws Exception { @Test public void testGetSessionProperties() throws Exception { final Executor executor = createDefaultExecutor(); - final SessionContext context = new SessionContext("test-session", new Environment()); + final SessionContext session = new SessionContext("test-session", new Environment()); // modify defaults - context.setSessionProperty("execution.result-mode", "table"); + session.setSessionProperty("execution.result-mode", "table"); - final Map actualProperties = executor.getSessionProperties(context); + final Map actualProperties = executor.getSessionProperties(session); final Map expectedProperties = new HashMap<>(); expectedProperties.put("execution.type", "streaming"); @@ -114,9 +115,9 @@ public void testGetSessionProperties() throws Exception { @Test public void testTableSchema() throws Exception { final Executor executor = createDefaultExecutor(); - final SessionContext context = new SessionContext("test-session", new Environment()); + final SessionContext session = new SessionContext("test-session", new Environment()); - final TableSchema actualTableSchema = executor.getTableSchema(context, "TableNumber2"); + final TableSchema actualTableSchema = executor.getTableSchema(session, "TableNumber2"); final TableSchema expectedTableSchema = new TableSchema( new String[] {"IntegerField2", "StringField2"}, @@ -135,11 +136,11 @@ public void testQueryExecutionChangelog() throws Exception { replaceVars.put("$VAR_2", "changelog"); final Executor executor = createModifiedExecutor(replaceVars); - final SessionContext context = new SessionContext("test-session", new Environment()); + final SessionContext session = new SessionContext("test-session", new Environment()); try { // start job and retrieval - final ResultDescriptor desc = executor.executeQuery(context, "SELECT * FROM TableNumber1"); + final ResultDescriptor desc = executor.executeQuery(session, "SELECT * FROM TableNumber1"); assertFalse(desc.isMaterialized()); @@ -148,7 +149,7 @@ public void testQueryExecutionChangelog() throws Exception { while (true) { Thread.sleep(50); // slow the processing down final TypedResult>> result = - executor.retrieveResultChanges(context, desc.getResultId()); + executor.retrieveResultChanges(session, desc.getResultId()); if (result.getType() == TypedResult.ResultType.PAYLOAD) { for (Tuple2 change : result.getPayload()) { actualResults.add(change.toString()); @@ -168,7 +169,7 @@ public void testQueryExecutionChangelog() throws Exception { TestBaseUtils.compareResultCollections(expectedResults, actualResults, Comparator.naturalOrder()); } finally { - executor.stop(context); + executor.stop(session); } } @@ -182,11 +183,11 @@ public void testQueryExecutionTable() throws Exception { replaceVars.put("$VAR_2", "table"); final Executor executor = createModifiedExecutor(replaceVars); - final SessionContext context = new SessionContext("test-session", new Environment()); + final SessionContext session = new SessionContext("test-session", new Environment()); try { // start job and retrieval - final ResultDescriptor desc = executor.executeQuery(context, "SELECT IntegerField1 FROM TableNumber1"); + final ResultDescriptor desc = executor.executeQuery(session, "SELECT IntegerField1 FROM TableNumber1"); assertTrue(desc.isMaterialized()); @@ -194,7 +195,7 @@ public void testQueryExecutionTable() throws Exception { while (true) { Thread.sleep(50); // slow the processing down - final TypedResult result = executor.snapshotResult(context, desc.getResultId(), 2); + final TypedResult result = executor.snapshotResult(session, desc.getResultId(), 2); if (result.getType() == TypedResult.ResultType.PAYLOAD) { actualResults.clear(); IntStream.rangeClosed(1, result.getPayload()).forEach((page) -> { @@ -217,29 +218,21 @@ public void testQueryExecutionTable() throws Exception { TestBaseUtils.compareResultCollections(expectedResults, actualResults, Comparator.naturalOrder()); } finally { - executor.stop(context); + executor.stop(session); } } private LocalExecutor createDefaultExecutor() throws Exception { - final URL url = getClass().getClassLoader().getResource("test-sql-client-defaults.yaml"); - Objects.requireNonNull(url); - final Environment env = Environment.parse(url); - - return new LocalExecutor(env, Collections.emptyList(), cluster.getConfiguration()); + return new LocalExecutor( + EnvironmentFileUtil.parseUnmodified(DEFAULTS_ENVIRONMENT_FILE), + Collections.emptyList(), + cluster.getConfiguration()); } private LocalExecutor createModifiedExecutor(Map replaceVars) throws Exception { - final URL url = getClass().getClassLoader().getResource("test-sql-client-defaults.yaml"); - Objects.requireNonNull(url); - String schema = FileUtils.readFileUtf8(new File(url.getFile())); - - for (Map.Entry replaceVar : replaceVars.entrySet()) { - schema = schema.replace(replaceVar.getKey(), replaceVar.getValue()); - } - - final Environment env = Environment.parse(schema); - - return new LocalExecutor(env, Collections.emptyList(), cluster.getConfiguration()); + return new LocalExecutor( + EnvironmentFileUtil.parseModified(DEFAULTS_ENVIRONMENT_FILE, replaceVars), + Collections.emptyList(), + cluster.getConfiguration()); } } diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/EnvironmentFileUtil.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/EnvironmentFileUtil.java new file mode 100644 index 0000000000000..4645b424e2d85 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/EnvironmentFileUtil.java @@ -0,0 +1,56 @@ +/* + * 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.table.client.gateway.utils; + +import org.apache.flink.table.client.config.Environment; +import org.apache.flink.util.FileUtils; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.Map; +import java.util.Objects; + +/** + * Utilities for reading an environment file. + */ +public final class EnvironmentFileUtil { + + private EnvironmentFileUtil() { + // private + } + + public static Environment parseUnmodified(String fileName) throws IOException { + final URL url = EnvironmentFileUtil.class.getClassLoader().getResource(fileName); + Objects.requireNonNull(url); + return Environment.parse(url); + } + + public static Environment parseModified(String fileName, Map replaceVars) throws IOException { + final URL url = EnvironmentFileUtil.class.getClassLoader().getResource(fileName); + Objects.requireNonNull(url); + String schema = FileUtils.readFileUtf8(new File(url.getFile())); + + for (Map.Entry replaceVar : replaceVars.entrySet()) { + schema = schema.replace(replaceVar.getKey(), replaceVar.getValue()); + } + + return Environment.parse(schema); + } +} diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java new file mode 100644 index 0000000000000..40a7e7bac6785 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java @@ -0,0 +1,112 @@ +/* + * 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.table.client.gateway.utils; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.Types; +import org.apache.flink.table.client.gateway.local.DependencyTest; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.table.sources.TableSource; +import org.apache.flink.table.sources.TableSourceFactory; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; + +/** + * Table source factory for testing the classloading in {@link DependencyTest}. + */ +public class TestTableSourceFactory implements TableSourceFactory { + + @Override + public Map requiredContext() { + final Map context = new HashMap<>(); + context.put(CONNECTOR_TYPE(), "test-table-source-factory"); + return context; + } + + @Override + public List supportedProperties() { + final List properties = new ArrayList<>(); + properties.add("connector.test-property"); + properties.add(SCHEMA() + ".#." + SCHEMA_TYPE()); + properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); + return properties; + } + + @Override + public TableSource create(Map properties) { + final DescriptorProperties params = new DescriptorProperties(true); + params.putProperties(properties); + return new TestTableSource( + params.getTableSchema(SCHEMA()), + properties.get("connector.test-property")); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Test table source. + */ + public static class TestTableSource implements StreamTableSource { + + private final TableSchema schema; + private final String property; + + public TestTableSource(TableSchema schema, String property) { + this.schema = schema; + this.property = property; + } + + public String getProperty() { + return property; + } + + @Override + public DataStream getDataStream(StreamExecutionEnvironment execEnv) { + return null; + } + + @Override + public TypeInformation getReturnType() { + return Types.ROW(schema.getColumnNames(), schema.getTypes()); + } + + @Override + public TableSchema getTableSchema() { + return schema; + } + + @Override + public String explainSource() { + return "TestTableSource"; + } + } +} diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file b/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file new file mode 100644 index 0000000000000..41e7fb2cbd561 --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/resources/test-factory-services-file @@ -0,0 +1,20 @@ +# 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. + +#============================================================================== +# Test file for org.apache.flink.table.client.gateway.local.DependencyTest. +#============================================================================== + +org.apache.flink.table.client.gateway.utils.TestTableSourceFactory diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml index 1c2a705ae57d4..9cbecb07903ab 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml @@ -18,6 +18,7 @@ #============================================================================== # TEST ENVIRONMENT FILE +# Intended for org.apache.flink.table.client.gateway.local.LocalExecutorITCase. #============================================================================== # this file has variables that can be filled with content by replacing $VAR_XXX diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml new file mode 100644 index 0000000000000..1bb69e537f40a --- /dev/null +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml @@ -0,0 +1,45 @@ +################################################################################ +# 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. +################################################################################ + +#============================================================================== +# TEST ENVIRONMENT FILE +# Intended for org.apache.flink.table.client.gateway.local.DependencyTest. +#============================================================================== + +# this file has variables that can be filled with content by replacing $VAR_XXX + +sources: + - name: TableNumber1 + schema: + - name: IntegerField1 + type: INT + - name: StringField1 + type: VARCHAR + connector: + type: "$VAR_0" + $VAR_1: "$VAR_2" + +execution: + type: streaming + parallelism: 1 + +deployment: + type: standalone + response-timeout: 5000 + + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Rowtime.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Rowtime.scala index ed3854df36b64..1e28303d0e48e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Rowtime.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/Rowtime.scala @@ -85,7 +85,7 @@ class Rowtime extends Descriptor { * * Emits watermarks which are the maximum observed timestamp minus the specified delay. */ - def watermarksPeriodicBounding(delay: Long): Rowtime = { + def watermarksPeriodicBounded(delay: Long): Rowtime = { watermarkStrategy = Some(new BoundedOutOfOrderTimestamps(delay)) this } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala index 877cb7b5f3912..0c81335f6b6fb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSourceFactoryService.scala @@ -36,18 +36,39 @@ import scala.collection.mutable */ object TableSourceFactoryService extends Logging { - private lazy val loader = ServiceLoader.load(classOf[TableSourceFactory[_]]) + private lazy val defaultLoader = ServiceLoader.load(classOf[TableSourceFactory[_]]) def findAndCreateTableSource(descriptor: TableSourceDescriptor): TableSource[_] = { + findAndCreateTableSource(descriptor, null) + } + + def findAndCreateTableSource( + descriptor: TableSourceDescriptor, + classLoader: ClassLoader) + : TableSource[_] = { + val properties = new DescriptorProperties() descriptor.addProperties(properties) - findAndCreateTableSource(properties.asMap.asScala.toMap) + findAndCreateTableSource(properties.asMap.asScala.toMap, classLoader) } def findAndCreateTableSource(properties: Map[String, String]): TableSource[_] = { + findAndCreateTableSource(properties, null) + } + + def findAndCreateTableSource( + properties: Map[String, String], + classLoader: ClassLoader) + : TableSource[_] = { + var matchingFactory: Option[(TableSourceFactory[_], Seq[String])] = None try { - val iter = loader.iterator() + val iter = if (classLoader == null) { + defaultLoader.iterator() + } else { + val customLoader = ServiceLoader.load(classOf[TableSourceFactory[_]], classLoader) + customLoader.iterator() + } while (iter.hasNext) { val factory = iter.next() diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala index 7968b481db343..9e339d02adc83 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala @@ -46,7 +46,7 @@ class RowtimeTest extends DescriptorTestBase { override def descriptors(): util.List[Descriptor] = { val desc1 = Rowtime() .timestampsFromField("otherField") - .watermarksPeriodicBounding(1000L) + .watermarksPeriodicBounded(1000L) val desc2 = Rowtime() .timestampsFromSource() From 8a5d0b229c0fbbbf4cd8d043bf661c816434a24f Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 6 Mar 2018 15:54:13 +0100 Subject: [PATCH 086/268] [hotfix] [taskmanager] Fix checkstyle in Task and TaskTest --- .../flink/runtime/taskmanager/Task.java | 80 +++++------ .../flink/runtime/taskmanager/TaskTest.java | 126 +++++++++--------- 2 files changed, 106 insertions(+), 100 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 1ecb47a6288d7..00fbdfefca68b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -127,10 +127,10 @@ public class Task implements Runnable, TaskActions, CheckpointListener { /** The class logger. */ private static final Logger LOG = LoggerFactory.getLogger(Task.class); - /** The tread group that contains all task threads */ + /** The tread group that contains all task threads. */ private static final ThreadGroup TASK_THREADS_GROUP = new ThreadGroup("Flink Task Threads"); - /** For atomic state updates */ + /** For atomic state updates. */ private static final AtomicReferenceFieldUpdater STATE_UPDATER = AtomicReferenceFieldUpdater.newUpdater(Task.class, ExecutionState.class, "executionState"); @@ -138,52 +138,52 @@ public class Task implements Runnable, TaskActions, CheckpointListener { // Constant fields that are part of the initial Task construction // ------------------------------------------------------------------------ - /** The job that the task belongs to */ + /** The job that the task belongs to. */ private final JobID jobId; - /** The vertex in the JobGraph whose code the task executes */ + /** The vertex in the JobGraph whose code the task executes. */ private final JobVertexID vertexId; - /** The execution attempt of the parallel subtask */ + /** The execution attempt of the parallel subtask. */ private final ExecutionAttemptID executionId; - /** ID which identifies the slot in which the task is supposed to run */ + /** ID which identifies the slot in which the task is supposed to run. */ private final AllocationID allocationId; - /** TaskInfo object for this task */ + /** TaskInfo object for this task. */ private final TaskInfo taskInfo; - /** The name of the task, including subtask indexes */ + /** The name of the task, including subtask indexes. */ private final String taskNameWithSubtask; - /** The job-wide configuration object */ + /** The job-wide configuration object. */ private final Configuration jobConfiguration; - /** The task-specific configuration */ + /** The task-specific configuration. */ private final Configuration taskConfiguration; - /** The jar files used by this task */ + /** The jar files used by this task. */ private final Collection requiredJarFiles; - /** The classpaths used by this task */ + /** The classpaths used by this task. */ private final Collection requiredClasspaths; - /** The name of the class that holds the invokable code */ + /** The name of the class that holds the invokable code. */ private final String nameOfInvokableClass; - /** Access to task manager configuration and host names*/ + /** Access to task manager configuration and host names. */ private final TaskManagerRuntimeInfo taskManagerConfig; - /** The memory manager to be used by this task */ + /** The memory manager to be used by this task. */ private final MemoryManager memoryManager; - /** The I/O manager to be used by this task */ + /** The I/O manager to be used by this task. */ private final IOManager ioManager; - /** The BroadcastVariableManager to be used by this task */ + /** The BroadcastVariableManager to be used by this task. */ private final BroadcastVariableManager broadcastVariableManager; - /** The manager for state of operators running in this task/slot */ + /** The manager for state of operators running in this task/slot. */ private final TaskStateManager taskStateManager; /** Serialized version of the job specific execution configuration (see {@link ExecutionConfig}). */ @@ -195,43 +195,43 @@ public class Task implements Runnable, TaskActions, CheckpointListener { private final Map inputGatesById; - /** Connection to the task manager */ + /** Connection to the task manager. */ private final TaskManagerActions taskManagerActions; - /** Input split provider for the task */ + /** Input split provider for the task. */ private final InputSplitProvider inputSplitProvider; - /** Checkpoint notifier used to communicate with the CheckpointCoordinator */ + /** Checkpoint notifier used to communicate with the CheckpointCoordinator. */ private final CheckpointResponder checkpointResponder; - /** All listener that want to be notified about changes in the task's execution state */ + /** All listener that want to be notified about changes in the task's execution state. */ private final List taskExecutionStateListeners; - /** The BLOB cache, from which the task can request BLOB files */ + /** The BLOB cache, from which the task can request BLOB files. */ private final BlobCacheService blobService; - /** The library cache, from which the task can request its class loader */ + /** The library cache, from which the task can request its class loader. */ private final LibraryCacheManager libraryCache; - /** The cache for user-defined files that the invokable requires */ + /** The cache for user-defined files that the invokable requires. */ private final FileCache fileCache; - /** The gateway to the network stack, which handles inputs and produced results */ + /** The gateway to the network stack, which handles inputs and produced results. */ private final NetworkEnvironment network; - /** The registry of this task which enables live reporting of accumulators */ + /** The registry of this task which enables live reporting of accumulators. */ private final AccumulatorRegistry accumulatorRegistry; - /** The thread that executes the task */ + /** The thread that executes the task. */ private final Thread executingThread; - /** Parent group for all metrics of this task */ + /** Parent group for all metrics of this task. */ private final TaskMetricGroup metrics; - /** Partition producer state checker to request partition states from */ + /** Partition producer state checker to request partition states from. */ private final PartitionProducerStateChecker partitionProducerStateChecker; - /** Executor to run future callbacks */ + /** Executor to run future callbacks. */ private final Executor executor; // ------------------------------------------------------------------------ @@ -240,19 +240,19 @@ public class Task implements Runnable, TaskActions, CheckpointListener { // proper happens-before semantics on parallel modification // ------------------------------------------------------------------------ - /** atomic flag that makes sure the invokable is canceled exactly once upon error */ + /** atomic flag that makes sure the invokable is canceled exactly once upon error. */ private final AtomicBoolean invokableHasBeenCanceled; - /** The invokable of this task, if initialized */ + /** The invokable of this task, if initialized. */ private volatile AbstractInvokable invokable; - /** The current execution state of the task */ + /** The current execution state of the task. */ private volatile ExecutionState executionState = ExecutionState.CREATED; - /** The observed exception, in case the task execution failed */ + /** The observed exception, in case the task execution failed. */ private volatile Throwable failureCause; - /** Serial executor for asynchronous calls (checkpoints, etc), lazily initialized */ + /** Serial executor for asynchronous calls (checkpoints, etc), lazily initialized. */ private volatile ExecutorService asyncCallDispatcher; /** Initialized from the Flink configuration. May also be set at the ExecutionConfig */ @@ -512,7 +512,7 @@ public void startTaskThread() { } /** - * The core work method that bootstraps the task and executes its code + * The core work method that bootstraps the task and executes its code. */ @Override public void run() { @@ -556,7 +556,7 @@ else if (current == ExecutionState.CANCELING) { // all resource acquisitions and registrations from here on // need to be undone in the end - Map> distributedCacheEntries = new HashMap>(); + Map> distributedCacheEntries = new HashMap<>(); AbstractInvokable invokable = null; try { @@ -743,8 +743,8 @@ else if (current == ExecutionState.CANCELING) { try { // check if the exception is unrecoverable if (ExceptionUtils.isJvmFatalError(t) || - (t instanceof OutOfMemoryError && taskManagerConfig.shouldExitJvmOnOutOfMemoryError())) - { + (t instanceof OutOfMemoryError && taskManagerConfig.shouldExitJvmOnOutOfMemoryError())) { + // terminate the JVM immediately // don't attempt a clean shutdown, because we cannot expect the clean shutdown to complete try { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 5b33d1922b13a..1829e977489eb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -29,8 +29,6 @@ import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.Executors; -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; -import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; @@ -103,12 +101,12 @@ * Tests for the Task, which make sure that correct state transitions happen, * and failures are correctly handled. * - * All tests here have a set of mock actors for TaskManager, JobManager, and + *

    All tests here have a set of mock actors for TaskManager, JobManager, and * execution listener, which simply put the messages in a queue to be picked * up by the test and validated. */ public class TaskTest extends TestLogger { - + private static OneShotLatch awaitLatch; private static OneShotLatch triggerLatch; private static OneShotLatch cancelLatch; @@ -123,7 +121,7 @@ public class TaskTest extends TestLogger { private BlockingQueue taskManagerMessages; private BlockingQueue jobManagerMessages; private BlockingQueue listenerMessages; - + @Before public void createQueuesAndActors() { taskManagerMessages = new LinkedBlockingQueue<>(); @@ -135,7 +133,7 @@ public void createQueuesAndActors() { listener = new ActorGatewayTaskExecutionStateListener(listenerGateway); taskManagerConnection = new ActorGatewayTaskManagerActions(taskManagerGateway); - + awaitLatch = new OneShotLatch(); triggerLatch = new OneShotLatch(); cancelLatch = new OneShotLatch(); @@ -155,32 +153,32 @@ public void clearActorsAndMessages() { // ------------------------------------------------------------------------ // Tests // ------------------------------------------------------------------------ - + @Test public void testRegularExecution() { try { Task task = createTask(TestInvokableCorrect.class); - + // task should be new and perfect assertEquals(ExecutionState.CREATED, task.getExecutionState()); assertFalse(task.isCanceledOrFailed()); assertNull(task.getFailureCause()); - + task.registerExecutionListener(listener); - + // go into the run method. we should switch to DEPLOYING, RUNNING, then // FINISHED, and all should be good task.run(); - + // verify final state assertEquals(ExecutionState.FINISHED, task.getExecutionState()); assertFalse(task.isCanceledOrFailed()); assertNull(task.getFailureCause()); - + // verify listener messages validateListenerMessage(ExecutionState.RUNNING, task, false); validateListenerMessage(ExecutionState.FINISHED, task, false); - + // make sure that the TaskManager received an message to unregister the task validateTaskManagerStateChange(ExecutionState.RUNNING, task, false); validateUnregisterTask(task.getExecutionId()); @@ -198,7 +196,7 @@ public void testCancelRightAway() { task.cancelExecution(); assertEquals(ExecutionState.CANCELING, task.getExecutionState()); - + task.run(); // verify final state @@ -274,7 +272,7 @@ public void testExecutionFailsInNetworkRegistration() { // mock a working library cache LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(getClass().getClassLoader()); - + // mock a network manager that rejects registration ResultPartitionManager partitionManager = mock(ResultPartitionManager.class); ResultPartitionConsumableNotifier consumableNotifier = mock(ResultPartitionConsumableNotifier.class); @@ -296,7 +294,7 @@ public void testExecutionFailsInNetworkRegistration() { assertEquals(ExecutionState.FAILED, task.getExecutionState()); assertTrue(task.isCanceledOrFailed()); assertTrue(task.getFailureCause().getMessage().contains("buffers")); - + validateUnregisterTask(task.getExecutionId()); validateListenerMessage(ExecutionState.FAILED, task, true); } @@ -326,13 +324,13 @@ public void testInvokableInstantiationFailed() { fail(e.getMessage()); } } - + @Test public void testExecutionFailsInInvoke() { try { Task task = createTask(InvokableWithExceptionInInvoke.class); task.registerExecutionListener(listener); - + task.run(); assertEquals(ExecutionState.FAILED, task.getExecutionState()); @@ -343,7 +341,7 @@ public void testExecutionFailsInInvoke() { validateTaskManagerStateChange(ExecutionState.RUNNING, task, false); validateUnregisterTask(task.getExecutionId()); - + validateListenerMessage(ExecutionState.RUNNING, task, false); validateListenerMessage(ExecutionState.FAILED, task, true); } @@ -378,7 +376,7 @@ public void testFailWithWrappedException() { fail(e.getMessage()); } } - + @Test public void testCancelDuringInvoke() { try { @@ -403,7 +401,7 @@ public void testCancelDuringInvoke() { validateTaskManagerStateChange(ExecutionState.RUNNING, task, false); validateUnregisterTask(task.getExecutionId()); - + validateListenerMessage(ExecutionState.RUNNING, task, false); validateCancelingAndCanceledListenerMessage(task); } @@ -463,7 +461,7 @@ public void testCanceledAfterExecutionFailedInInvoke() { validateTaskManagerStateChange(ExecutionState.RUNNING, task, false); validateUnregisterTask(task.getExecutionId()); - + validateListenerMessage(ExecutionState.RUNNING, task, false); validateListenerMessage(ExecutionState.FAILED, task, true); } @@ -472,7 +470,7 @@ public void testCanceledAfterExecutionFailedInInvoke() { fail(e.getMessage()); } } - + @Test public void testExecutionFailesAfterCanceling() { try { @@ -487,7 +485,7 @@ public void testExecutionFailesAfterCanceling() { task.cancelExecution(); assertEquals(ExecutionState.CANCELING, task.getExecutionState()); - + // this causes an exception triggerLatch.trigger(); @@ -497,7 +495,7 @@ public void testExecutionFailesAfterCanceling() { assertEquals(ExecutionState.CANCELED, task.getExecutionState()); assertTrue(task.isCanceledOrFailed()); assertNull(task.getFailureCause()); - + validateTaskManagerStateChange(ExecutionState.RUNNING, task, false); validateUnregisterTask(task.getExecutionId()); @@ -529,11 +527,11 @@ public void testExecutionFailsAfterTaskMarkedFailed() { triggerLatch.trigger(); task.getExecutingThread().join(); - + assertEquals(ExecutionState.FAILED, task.getExecutionState()); assertTrue(task.isCanceledOrFailed()); assertTrue(task.getFailureCause().getMessage().contains("external")); - + validateTaskManagerStateChange(ExecutionState.RUNNING, task, false); validateUnregisterTask(task.getExecutionId()); @@ -606,7 +604,7 @@ public void testOnPartitionStateUpdate() throws Exception { expected.put(ExecutionState.SCHEDULED, ExecutionState.RUNNING); expected.put(ExecutionState.DEPLOYING, ExecutionState.RUNNING); expected.put(ExecutionState.FINISHED, ExecutionState.RUNNING); - + expected.put(ExecutionState.CANCELED, ExecutionState.CANCELING); expected.put(ExecutionState.CANCELING, ExecutionState.CANCELING); expected.put(ExecutionState.FAILED, ExecutionState.CANCELING); @@ -901,10 +899,9 @@ private void setState(Task task, ExecutionState state) { * @return BlobCache mock with the bare minimum of implemented functions that work */ private BlobCacheService createBlobCache() { - BlobCacheService blobService = - new BlobCacheService(mock(PermanentBlobCache.class), mock(TransientBlobCache.class)); - - return blobService; + return new BlobCacheService( + mock(PermanentBlobCache.class), + mock(TransientBlobCache.class)); } private Task createTask(Class invokable) throws IOException { @@ -915,14 +912,14 @@ private Task createTask(Class invokable, Configurat BlobCacheService blobService = createBlobCache(); LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(getClass().getClassLoader()); - return createTask(invokable, blobService,libCache, config, new ExecutionConfig()); + return createTask(invokable, blobService, libCache, config, new ExecutionConfig()); } private Task createTask(Class invokable, Configuration config, ExecutionConfig execConfig) throws IOException { BlobCacheService blobService = createBlobCache(); LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(getClass().getClassLoader()); - return createTask(invokable, blobService,libCache, config, execConfig); + return createTask(invokable, blobService, libCache, config, execConfig); } private Task createTask( @@ -930,7 +927,7 @@ private Task createTask( BlobCacheService blobService, LibraryCacheManager libCache) throws IOException { - return createTask(invokable, blobService,libCache, new Configuration(), new ExecutionConfig()); + return createTask(invokable, blobService, libCache, new Configuration(), new ExecutionConfig()); } private Task createTask( @@ -965,7 +962,7 @@ private Task createTask( Executor executor) throws IOException { return createTask(invokable, blobService, libCache, networkEnvironment, consumableNotifier, partitionProducerStateChecker, executor, new Configuration(), new ExecutionConfig()); } - + private Task createTask( Class invokable, BlobCacheService blobService, @@ -976,7 +973,7 @@ private Task createTask( Executor executor, Configuration taskManagerConfig, ExecutionConfig execConfig) throws IOException { - + JobID jobId = new JobID(); JobVertexID jobVertexId = new JobVertexID(); ExecutionAttemptID executionAttemptId = new ExecutionAttemptID(); @@ -1010,7 +1007,7 @@ private Task createTask( TaskMetricGroup taskMetricGroup = mock(TaskMetricGroup.class); when(taskMetricGroup.getIOMetricGroup()).thenReturn(mock(TaskIOMetricGroup.class)); - + return new Task( jobInformation, taskInformation, @@ -1018,8 +1015,8 @@ private Task createTask( new AllocationID(), 0, 0, - Collections.emptyList(), - Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), 0, mock(MemoryManager.class), mock(IOManager.class), @@ -1042,18 +1039,18 @@ private Task createTask( // ------------------------------------------------------------------------ // Validation Methods // ------------------------------------------------------------------------ - + private void validateUnregisterTask(ExecutionAttemptID id) { try { // we may have to wait for a bit to give the actors time to receive the message // and put it into the queue Object rawMessage = taskManagerMessages.take(); - + assertNotNull("There is no additional TaskManager message", rawMessage); if (!(rawMessage instanceof TaskMessages.TaskInFinalState)) { fail("TaskManager message is not 'UnregisterTask', but " + rawMessage.getClass()); } - + TaskMessages.TaskInFinalState message = (TaskMessages.TaskInFinalState) rawMessage; assertEquals(id, message.executionID()); } @@ -1072,10 +1069,10 @@ private void validateTaskManagerStateChange(ExecutionState state, Task task, boo if (!(rawMessage instanceof TaskMessages.UpdateTaskExecutionState)) { fail("TaskManager message is not 'UpdateTaskExecutionState', but " + rawMessage.getClass()); } - + TaskMessages.UpdateTaskExecutionState message = (TaskMessages.UpdateTaskExecutionState) rawMessage; - + TaskExecutionState taskState = message.taskExecutionState(); assertEquals(task.getJobID(), taskState.getJobID()); @@ -1092,7 +1089,7 @@ private void validateTaskManagerStateChange(ExecutionState state, Task task, boo fail("interrupted"); } } - + private void validateListenerMessage(ExecutionState state, Task task, boolean hasError) { try { // we may have to wait for a bit to give the actors time to receive the message @@ -1100,13 +1097,13 @@ private void validateListenerMessage(ExecutionState state, Task task, boolean ha TaskMessages.UpdateTaskExecutionState message = (TaskMessages.UpdateTaskExecutionState) listenerMessages.take(); assertNotNull("There is no additional listener message", message); - + TaskExecutionState taskState = message.taskExecutionState(); assertEquals(task.getJobID(), taskState.getJobID()); assertEquals(task.getExecutionId(), taskState.getID()); assertEquals(state, taskState.getExecutionState()); - + if (hasError) { assertNotNull(taskState.getError(getClass().getClassLoader())); } else { @@ -1126,8 +1123,7 @@ private void validateCancelingAndCanceledListenerMessage(Task task) { (TaskMessages.UpdateTaskExecutionState) listenerMessages.take(); TaskMessages.UpdateTaskExecutionState message2 = (TaskMessages.UpdateTaskExecutionState) listenerMessages.take(); - - + assertNotNull("There is no additional listener message", message1); assertNotNull("There is no additional listener message", message2); @@ -1138,30 +1134,31 @@ private void validateCancelingAndCanceledListenerMessage(Task task) { assertEquals(task.getJobID(), taskState2.getJobID()); assertEquals(task.getExecutionId(), taskState1.getID()); assertEquals(task.getExecutionId(), taskState2.getID()); - + ExecutionState state1 = taskState1.getExecutionState(); ExecutionState state2 = taskState2.getExecutionState(); - + // it may be (very rarely) that the following race happens: // - OUTSIDE THREAD: call to cancel() // - OUTSIDE THREAD: atomic state change from running to canceling // - TASK THREAD: finishes, atomic change from canceling to canceled // - TASK THREAD: send notification that state is canceled // - OUTSIDE THREAD: send notification that state is canceling - + // for that reason, we allow the notification messages in any order. - assertTrue( (state1 == ExecutionState.CANCELING && state2 == ExecutionState.CANCELED) || + assertTrue((state1 == ExecutionState.CANCELING && state2 == ExecutionState.CANCELED) || (state2 == ExecutionState.CANCELING && state1 == ExecutionState.CANCELED)); } catch (InterruptedException e) { fail("interrupted"); } } - + // -------------------------------------------------------------------------------------------- // Mock invokable code // -------------------------------------------------------------------------------------------- - + + /** Test task class. */ public static final class TestInvokableCorrect extends AbstractInvokable { public TestInvokableCorrect(Environment environment) { @@ -1177,6 +1174,7 @@ public void cancel() throws Exception { } } + /** Test task class. */ public static final class InvokableWithExceptionInInvoke extends AbstractInvokable { public InvokableWithExceptionInInvoke(Environment environment) { @@ -1189,6 +1187,7 @@ public void invoke() throws Exception { } } + /** Test task class. */ public static final class InvokableWithExceptionOnTrigger extends AbstractInvokable { public InvokableWithExceptionOnTrigger(Environment environment) { @@ -1198,7 +1197,7 @@ public InvokableWithExceptionOnTrigger(Environment environment) { @Override public void invoke() { awaitLatch.trigger(); - + // make sure that the interrupt call does not // grab us out of the lock early while (true) { @@ -1215,13 +1214,15 @@ public void invoke() { } } - public static abstract class InvokableNonInstantiable extends AbstractInvokable { + /** Test task class. */ + public abstract static class InvokableNonInstantiable extends AbstractInvokable { public InvokableNonInstantiable(Environment environment) { super(environment); } } + /** Test task class. */ public static final class InvokableBlockingInInvoke extends AbstractInvokable { public InvokableBlockingInInvoke(Environment environment) { @@ -1231,7 +1232,7 @@ public InvokableBlockingInInvoke(Environment environment) { @Override public void invoke() throws Exception { awaitLatch.trigger(); - + // block forever synchronized (this) { wait(); @@ -1239,6 +1240,7 @@ public void invoke() throws Exception { } } + /** Test task class. */ public static final class InvokableWithCancelTaskExceptionInInvoke extends AbstractInvokable { public InvokableWithCancelTaskExceptionInInvoke(Environment environment) { @@ -1253,11 +1255,12 @@ public void invoke() throws Exception { triggerLatch.await(); } catch (Throwable ignored) {} - + throw new CancelTaskException(); } } + /** Test task class. */ public static final class InvokableInterruptableSharedLockInInvokeAndCancel extends AbstractInvokable { private final Object lock = new Object(); @@ -1282,6 +1285,7 @@ public void cancel() throws Exception { } } + /** Test task class. */ public static final class InvokableBlockingInCancel extends AbstractInvokable { public InvokableBlockingInCancel(Environment environment) { @@ -1313,6 +1317,7 @@ public void cancel() throws Exception { } } + /** Test task class. */ public static final class InvokableUninterruptibleBlockingInvoke extends AbstractInvokable { public InvokableUninterruptibleBlockingInvoke(Environment environment) { @@ -1337,6 +1342,7 @@ public void cancel() throws Exception { } } + /** Test task class. */ public static final class FailingInvokableWithChainedException extends AbstractInvokable { public FailingInvokableWithChainedException(Environment environment) { From 36dccdc520c676ca2c53046cad4ec80c351068b7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 6 Mar 2018 15:18:33 +0100 Subject: [PATCH 087/268] [FLINK-8856] [TaskManager] Move all cancellation interrupt calls to TaskCanceller thread This cleans up the code and guards against a JVM bug where 'interrupt()' calls block/deadlock if the thread is engaged in certain I/O operations. In addition, this makes sure that the process really goes away when the cancellation timeout expires, rather than relying on the TaskManager to be able to properly handle the fatal error notification. --- .../flink/runtime/taskmanager/Task.java | 280 +++++++++++------- 1 file changed, 177 insertions(+), 103 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 00fbdfefca68b..ccb850e85bc12 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -67,6 +67,7 @@ import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.runtime.state.TaskStateManager; +import org.apache.flink.runtime.util.FatalExitExceptionHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; @@ -77,7 +78,6 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; -import javax.annotation.Nullable; import java.io.IOException; import java.lang.reflect.Constructor; @@ -94,11 +94,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -1048,15 +1048,51 @@ else if (current == ExecutionState.RUNNING) { invokable, executingThread, taskNameWithSubtask, - taskCancellationInterval, - taskCancellationTimeout, - taskManagerActions, producedPartitions, inputGates); - Thread cancelThread = new Thread(executingThread.getThreadGroup(), canceler, + + Thread cancelThread = new Thread( + executingThread.getThreadGroup(), + canceler, String.format("Canceler for %s (%s).", taskNameWithSubtask, executionId)); cancelThread.setDaemon(true); + cancelThread.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE); cancelThread.start(); + + // the periodic interrupting thread - a different thread than the canceller, in case + // the application code does blocking stuff in its cancellation paths. + Runnable interrupter = new TaskInterrupter( + LOG, + executingThread, + taskNameWithSubtask, + taskCancellationInterval); + + Thread interruptingThread = new Thread( + executingThread.getThreadGroup(), + interrupter, + String.format("Canceler/Interrupts for %s (%s).", taskNameWithSubtask, executionId)); + interruptingThread.setDaemon(true); + interruptingThread.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE); + interruptingThread.start(); + + // if a cancellation timeout is set, the watchdog thread kills the process + // if graceful cancellation does not succeed + if (taskCancellationTimeout > 0) { + Runnable cancelWatchdog = new TaskCancelerWatchDog( + executingThread, + taskManagerActions, + taskCancellationTimeout, + LOG); + + Thread watchDogThread = new Thread( + executingThread.getThreadGroup(), + cancelWatchdog, + String.format("Cancellation Watchdog for %s (%s).", + taskNameWithSubtask, executionId)); + watchDogThread.setDaemon(true); + watchDogThread.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE); + watchDogThread.start(); + } } return; } @@ -1408,12 +1444,28 @@ private static AbstractInvokable loadAndInstantiateInvokable( } // ------------------------------------------------------------------------ - // TaskCanceler + // Task cancellation + // + // The task cancellation uses in total three threads, as a safety net + // against various forms of user- and JVM bugs. + // + // - The first thread calls 'cancel()' on the invokable and closes + // the input and output connections, for fast thread termination + // - The second thread periodically interrupts the invokable in order + // to pull the thread out of blocking wait and I/O operations + // - The third thread (watchdog thread) waits until the cancellation + // timeout and then performs a hard cancel (kill process, or let + // the TaskManager know) + // + // Previously, thread two and three were in one thread, but we needed + // to separate this to make sure the watchdog thread does not call + // 'interrupt()'. This is a workaround for the following JVM bug + // https://bugs.java.com/bugdatabase/view_bug.do?bug_id=8138622 // ------------------------------------------------------------------------ /** - * This runner calls cancel() on the invokable and periodically interrupts the - * thread until it has terminated. + * This runner calls cancel() on the invokable, closes input-/output resources, + * and initially interrupts the task thread. */ private static class TaskCanceler implements Runnable { @@ -1424,27 +1476,11 @@ private static class TaskCanceler implements Runnable { private final ResultPartition[] producedPartitions; private final SingleInputGate[] inputGates; - /** Interrupt interval. */ - private final long interruptInterval; - - /** Timeout after which a fatal error notification happens. */ - private final long interruptTimeout; - - /** TaskManager to notify about a timeout */ - private final TaskManagerActions taskManager; - - /** Watch Dog thread */ - @Nullable - private final Thread watchDogThread; - public TaskCanceler( Logger logger, AbstractInvokable invokable, Thread executer, String taskName, - long cancellationInterval, - long cancellationTimeout, - TaskManagerActions taskManager, ResultPartition[] producedPartitions, SingleInputGate[] inputGates) { @@ -1452,39 +1488,19 @@ public TaskCanceler( this.invokable = invokable; this.executer = executer; this.taskName = taskName; - this.interruptInterval = cancellationInterval; - this.interruptTimeout = cancellationTimeout; - this.taskManager = taskManager; this.producedPartitions = producedPartitions; this.inputGates = inputGates; - - if (cancellationTimeout > 0) { - // The watch dog repeatedly interrupts the executor until - // the cancellation timeout kicks in (at which point the - // task manager is notified about a fatal error) or the - // executor has terminated. - this.watchDogThread = new Thread( - executer.getThreadGroup(), - new TaskCancelerWatchDog(), - "WatchDog for " + taskName + " cancellation"); - this.watchDogThread.setDaemon(true); - } else { - this.watchDogThread = null; - } } @Override public void run() { try { - if (watchDogThread != null) { - watchDogThread.start(); - } - // the user-defined cancel method may throw errors. // we need do continue despite that try { invokable.cancel(); } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); logger.error("Error while canceling the task {}.", taskName, t); } @@ -1499,6 +1515,7 @@ public void run() { try { partition.destroyBufferPool(); } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); LOG.error("Failed to release result partition buffer pool for task {}.", taskName, t); } } @@ -1507,89 +1524,146 @@ public void run() { try { inputGate.releaseAllResources(); } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); LOG.error("Failed to release input gate for task {}.", taskName, t); } } - // interrupt the running thread initially + // send the initial interruption signal executer.interrupt(); - try { - executer.join(interruptInterval); - } - catch (InterruptedException e) { - // we can ignore this - } - - if (watchDogThread != null) { - watchDogThread.interrupt(); - watchDogThread.join(); - } - } catch (Throwable t) { + } + catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); logger.error("Error in the task canceler for task {}.", taskName, t); } } + } + + /** + * This thread sends the delayed, periodic interrupt calls to the executing thread. + */ + private static final class TaskInterrupter implements Runnable { - /** - * Watchdog for the cancellation. If the task is stuck in cancellation, - * we notify the task manager about a fatal error. - */ - private class TaskCancelerWatchDog implements Runnable { + /** The logger to report on the fatal condition. */ + private final Logger log; - @Override - public void run() { - long intervalNanos = TimeUnit.NANOSECONDS.convert(interruptInterval, TimeUnit.MILLISECONDS); - long timeoutNanos = TimeUnit.NANOSECONDS.convert(interruptTimeout, TimeUnit.MILLISECONDS); - long deadline = System.nanoTime() + timeoutNanos; + /** The executing task thread that we wait for to terminate. */ + private final Thread executerThread; - try { - // Initial wait before interrupting periodically - Thread.sleep(interruptInterval); - } catch (InterruptedException ignored) { - } + /** The name of the task, for logging purposes. */ + private final String taskName; - // It is possible that the user code does not react to the task canceller. - // for that reason, we spawn this separate thread that repeatedly interrupts - // the user code until it exits. If the user code does not exit within - // the timeout, we notify the job manager about a fatal error. - while (executer.isAlive()) { - long now = System.nanoTime(); + /** The interval in which we interrupt. */ + private final long interruptIntervalMillis; + + TaskInterrupter( + Logger log, + Thread executerThread, + String taskName, + long interruptIntervalMillis) { + + this.log = log; + this.executerThread = executerThread; + this.taskName = taskName; + this.interruptIntervalMillis = interruptIntervalMillis; + } + @Override + public void run() { + try { + // we initially wait for one interval + // in most cases, the threads go away immediately (by the cancellation thread) + // and we need not actually do anything + executerThread.join(interruptIntervalMillis); + + // log stack trace where the executing thread is stuck and + // interrupt the running thread periodically while it is still alive + while (executerThread.isAlive()) { // build the stack trace of where the thread is stuck, for the log + StackTraceElement[] stack = executerThread.getStackTrace(); StringBuilder bld = new StringBuilder(); - StackTraceElement[] stack = executer.getStackTrace(); for (StackTraceElement e : stack) { bld.append(e).append('\n'); } - if (now >= deadline) { - long duration = TimeUnit.SECONDS.convert(interruptInterval, TimeUnit.MILLISECONDS); - String msg = String.format("Task '%s' did not react to cancelling signal in " + - "the last %d seconds, but is stuck in method:\n %s", - taskName, - duration, - bld.toString()); + log.warn("Task '{}' did not react to cancelling signal for {} seconds, but is stuck in method:\n {}", + taskName, (interruptIntervalMillis / 1000), bld); - logger.info("Notifying TaskManager about fatal error. {}.", msg); + executerThread.interrupt(); + try { + executerThread.join(interruptIntervalMillis); + } + catch (InterruptedException e) { + // we ignore this and fall through the loop + } + } + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); + log.error("Error in the task canceler for task {}.", taskName, t); + } + } + } - taskManager.notifyFatalError(msg, null); + /** + * Watchdog for the cancellation. + * If the task thread does not go away gracefully within a certain time, we + * trigger a hard cancel action (notify TaskManager of fatal error, which in + * turn kills the process). + */ + private static class TaskCancelerWatchDog implements Runnable { - return; // done, don't forget to leave the loop - } else { - logger.warn("Task '{}' did not react to cancelling signal, but is stuck in method:\n {}", - taskName, bld.toString()); + /** The logger to report on the fatal condition. */ + private final Logger log; - executer.interrupt(); - try { - long timeLeftNanos = Math.min(intervalNanos, deadline - now); - long timeLeftMillis = TimeUnit.MILLISECONDS.convert(timeLeftNanos, TimeUnit.NANOSECONDS); + /** The executing task thread that we wait for to terminate. */ + private final Thread executerThread; - if (timeLeftMillis > 0) { - executer.join(timeLeftMillis); - } - } catch (InterruptedException ignored) { - } + /** The TaskManager to notify if cancellation does not happen in time. */ + private final TaskManagerActions taskManager; + + /** The timeout for cancellation. */ + private final long timeoutMillis; + + TaskCancelerWatchDog( + Thread executerThread, + TaskManagerActions taskManager, + long timeoutMillis, + Logger log) { + + checkArgument(timeoutMillis > 0); + + this.log = log; + this.executerThread = executerThread; + this.taskManager = taskManager; + this.timeoutMillis = timeoutMillis; + } + + @Override + public void run() { + try { + final long hardKillDeadline = System.nanoTime() + timeoutMillis * 1_000_000; + + long millisLeft; + while (executerThread.isAlive() + && (millisLeft = (hardKillDeadline - System.nanoTime()) / 1_000_000) > 0) { + + try { + executerThread.join(millisLeft); + } + catch (InterruptedException ignored) { + // we don't react to interrupted exceptions, simply fall through the loop } } + + if (executerThread.isAlive()) { + String msg = "Task did not exit gracefully within " + (timeoutMillis / 1000) + " + seconds."; + log.error(msg); + taskManager.notifyFatalError(msg, null); + } + } + catch (Throwable t) { + ExceptionUtils.rethrowIfFatalError(t); + log.error("Error in Task Cancellation Watch Dog", t); } } } From afd2404d634a03cc4cd017b63745c74e5281896c Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 6 Mar 2018 16:36:13 +0100 Subject: [PATCH 088/268] [FLINK-8883] [core] Make ThreadDeath a fatal error in ExceptionUtils --- .../src/main/java/org/apache/flink/util/ExceptionUtils.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java index b9a21ae349528..6af16fcfa4f6c 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java @@ -81,12 +81,15 @@ public static String stringifyException(final Throwable e) { *

    Currently considered fatal exceptions are Virtual Machine errors indicating * that the JVM is corrupted, like {@link InternalError}, {@link UnknownError}, * and {@link java.util.zip.ZipError} (a special case of InternalError). + * The {@link ThreadDeath} exception is also treated as a fatal error, because when + * a thread is forcefully stopped, there is a high chance that parts of the system + * are in an inconsistent state. * * @param t The exception to check. * @return True, if the exception is considered fatal to the JVM, false otherwise. */ public static boolean isJvmFatalError(Throwable t) { - return (t instanceof InternalError) || (t instanceof UnknownError); + return (t instanceof InternalError) || (t instanceof UnknownError) || (t instanceof ThreadDeath); } /** From 071e29143c303d63e9908fb1ef6faa447c48d27e Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 6 Mar 2018 17:14:54 +0100 Subject: [PATCH 089/268] [FLINK-8885] [TaskManager] DispatcherThreadFactory registers a fatal error exception handler In case dispatcher threads let an exception bubble out (do not handle the exception), the exception handler terminates the process, to ensure we don't leave broken processes. --- .../runtime/taskmanager/DispatcherThreadFactory.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/DispatcherThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/DispatcherThreadFactory.java index 543b15929e1de..9cec52402b06c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/DispatcherThreadFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/DispatcherThreadFactory.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.taskmanager; +import org.apache.flink.runtime.util.FatalExitExceptionHandler; + import javax.annotation.Nullable; import java.util.concurrent.ThreadFactory; @@ -27,13 +29,14 @@ * thread group, and set them to daemon mode. */ public class DispatcherThreadFactory implements ThreadFactory { - + private final ThreadGroup group; - + private final String threadName; + @Nullable private final ClassLoader classLoader; - + /** * Creates a new thread factory. * @@ -67,6 +70,7 @@ public Thread newThread(Runnable r) { t.setContextClassLoader(classLoader); } t.setDaemon(true); + t.setUncaughtExceptionHandler(FatalExitExceptionHandler.INSTANCE); return t; } } From a42616699ddf5290553e0423a7501084498ecab9 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 6 Mar 2018 17:18:38 +0100 Subject: [PATCH 090/268] [hotfix] [runtime] Harden FatalExitExceptionHandler In case the logging framework throws an exception when handling the exception, we still kill the process, as intended. --- .../flink/runtime/util/FatalExitExceptionHandler.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/FatalExitExceptionHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/FatalExitExceptionHandler.java index c57b75a452fa4..e162a580cbe12 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/FatalExitExceptionHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/FatalExitExceptionHandler.java @@ -34,9 +34,14 @@ public final class FatalExitExceptionHandler implements Thread.UncaughtException public static final FatalExitExceptionHandler INSTANCE = new FatalExitExceptionHandler(); @Override + @SuppressWarnings("finally") public void uncaughtException(Thread t, Throwable e) { - LOG.error("FATAL: Thread '" + t.getName() + - "' produced an uncaught exception. Stopping the process...", e); - System.exit(-17); + try { + LOG.error("FATAL: Thread '" + t.getName() + + "' produced an uncaught exception. Stopping the process...", e); + } + finally { + System.exit(-17); + } } } From 9125687635e403c11408dcc00a0bd4cb25432453 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Mar 2018 15:02:05 +0100 Subject: [PATCH 091/268] [FLINK-8887][tests] Add single retry in MiniClusterClient This closes #5657. --- .../client/program/MiniClusterClient.java | 32 +++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index dd99f0dabeb99..bbb5d49cbb8d3 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -25,11 +25,17 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; +import org.apache.flink.runtime.rpc.exceptions.FencingTokenException; +import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.util.FlinkException; @@ -43,6 +49,9 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.Supplier; /** * Client to interact with a {@link MiniCluster}. @@ -50,6 +59,8 @@ public class MiniClusterClient extends ClusterClient { private final MiniCluster miniCluster; + private final ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(4, new ExecutorThreadFactory("Flink-MiniClusterClient")); + private final ScheduledExecutor scheduledExecutor = new ScheduledExecutorServiceAdapter(scheduledExecutorService); public MiniClusterClient(@Nonnull Configuration configuration, @Nonnull MiniCluster miniCluster) throws Exception { super(configuration, miniCluster.getHighAvailabilityServices(), true); @@ -57,6 +68,12 @@ public MiniClusterClient(@Nonnull Configuration configuration, @Nonnull MiniClus this.miniCluster = miniCluster; } + @Override + public void shutdown() throws Exception { + super.shutdown(); + scheduledExecutorService.shutdown(); + } + @Override public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) throws ProgramInvocationException { if (isDetached()) { @@ -82,12 +99,12 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) @Override public void cancel(JobID jobId) throws Exception { - miniCluster.cancelJob(jobId); + guardWithSingleRetry(() -> miniCluster.cancelJob(jobId), scheduledExecutor); } @Override public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirectory) throws Exception { - return miniCluster.triggerSavepoint(jobId, savepointDirectory, true).get(); + return guardWithSingleRetry(() -> miniCluster.triggerSavepoint(jobId, savepointDirectory, true), scheduledExecutor).get(); } @Override @@ -121,7 +138,7 @@ public Map getAccumulators(JobID jobID, ClassLoader loader) thro } public CompletableFuture getJobStatus(JobID jobId) { - return miniCluster.getJobStatus(jobId); + return guardWithSingleRetry(() -> miniCluster.getJobStatus(jobId), scheduledExecutor); } @Override @@ -173,4 +190,13 @@ public boolean hasUserJarsInClassPath(List userJarFiles) { enum MiniClusterId { INSTANCE } + + private static CompletableFuture guardWithSingleRetry(Supplier> operation, ScheduledExecutor executor) { + return FutureUtils.retryWithDelay( + operation, + 1, + Time.milliseconds(500), + throwable -> throwable instanceof FencingTokenException || throwable instanceof AkkaRpcException, + executor); + } } From 56f2fdf369d0e67bac816883c03697691d43f714 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 5 Mar 2018 13:45:33 +0100 Subject: [PATCH 092/268] [hotfix][tests] Do not use singleActorSystem in LocalFlinkMiniCluster Using a singleActorSystem rendered the returned client unusable. This closes #5652. --- .../flink/test/util/MiniClusterResource.java | 43 ++++++++++++++----- 1 file changed, 33 insertions(+), 10 deletions(-) diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java index 954b06f65a9ca..66cbb9f9b7700 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java @@ -62,6 +62,8 @@ public class MiniClusterResource extends ExternalResource { private JobExecutorService jobExecutorService; + private final boolean enableClusterClient; + private ClusterClient clusterClient; private int numberSlots = -1; @@ -69,16 +71,25 @@ public class MiniClusterResource extends ExternalResource { private TestEnvironment executionEnvironment; public MiniClusterResource(final MiniClusterResourceConfiguration miniClusterResourceConfiguration) { + this(miniClusterResourceConfiguration, false); + } + + public MiniClusterResource( + final MiniClusterResourceConfiguration miniClusterResourceConfiguration, + final boolean enableClusterClient) { this( miniClusterResourceConfiguration, - Objects.equals(FLIP6_CODEBASE, System.getProperty(CODEBASE_KEY)) ? MiniClusterType.FLIP6 : MiniClusterType.OLD); + Objects.equals(FLIP6_CODEBASE, System.getProperty(CODEBASE_KEY)) ? MiniClusterType.FLIP6 : MiniClusterType.OLD, + enableClusterClient); } - public MiniClusterResource( + private MiniClusterResource( final MiniClusterResourceConfiguration miniClusterResourceConfiguration, - final MiniClusterType miniClusterType) { + final MiniClusterType miniClusterType, + final boolean enableClusterClient) { this.miniClusterResourceConfiguration = Preconditions.checkNotNull(miniClusterResourceConfiguration); this.miniClusterType = Preconditions.checkNotNull(miniClusterType); + this.enableClusterClient = enableClusterClient; } public int getNumberSlots() { @@ -86,6 +97,12 @@ public int getNumberSlots() { } public ClusterClient getClusterClient() { + if (!enableClusterClient) { + // this check is technically only necessary for legacy clusters + // we still fail here for flip6 to keep the behaviors in sync + throw new IllegalStateException("To use the client you must enable it with the constructor."); + } + return clusterClient; } @@ -113,10 +130,12 @@ public void after() { Exception exception = null; - try { - clusterClient.shutdown(); - } catch (Exception e) { - exception = e; + if (clusterClient != null) { + try { + clusterClient.shutdown(); + } catch (Exception e) { + exception = e; + } } clusterClient = null; @@ -158,10 +177,12 @@ private void startOldMiniCluster() throws Exception { final LocalFlinkMiniCluster flinkMiniCluster = TestBaseUtils.startCluster( configuration, - true); + !enableClusterClient); // the cluster client only works if separate actor systems are used jobExecutorService = flinkMiniCluster; - clusterClient = new StandaloneClusterClient(configuration, flinkMiniCluster.highAvailabilityServices(), true); + if (enableClusterClient) { + clusterClient = new StandaloneClusterClient(configuration, flinkMiniCluster.highAvailabilityServices(), true); + } } private void startFlip6MiniCluster() throws Exception { @@ -188,7 +209,9 @@ private void startFlip6MiniCluster() throws Exception { configuration.setInteger(RestOptions.REST_PORT, miniCluster.getRestAddress().getPort()); jobExecutorService = miniCluster; - clusterClient = new MiniClusterClient(configuration, miniCluster); + if (enableClusterClient) { + clusterClient = new MiniClusterClient(configuration, miniCluster); + } } /** From 15ffb18bfc2a41dfbb9f367077e3b07eb5665d5f Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Mar 2018 10:18:03 +0100 Subject: [PATCH 093/268] [FLINK-8889][tests] Do not override cluster config values This closes #5651. --- .../flink/test/util/MiniClusterResource.java | 4 +- .../apache/flink/test/util/TestBaseUtils.java | 43 +++++++++++++------ 2 files changed, 34 insertions(+), 13 deletions(-) diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java index 66cbb9f9b7700..2f12bdc347e84 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java @@ -190,7 +190,9 @@ private void startFlip6MiniCluster() throws Exception { // we need to set this since a lot of test expect this because TestBaseUtils.startCluster() // enabled this by default - configuration.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); + if (!configuration.contains(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE)) { + configuration.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); + } // set rest port to 0 to avoid clashes with concurrent MiniClusters configuration.setInteger(RestOptions.REST_PORT, 0); diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java index d7142f5fbfa18..dd255fd6dc2fd 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java @@ -143,22 +143,41 @@ public static LocalFlinkMiniCluster startCluster( Configuration config, boolean singleActorSystem) throws Exception { - logDir = File.createTempFile("TestBaseUtils-logdir", null); - Assert.assertTrue("Unable to delete temp file", logDir.delete()); - Assert.assertTrue("Unable to create temp directory", logDir.mkdir()); - Path logFile = Files.createFile(new File(logDir, "jobmanager.log").toPath()); - Files.createFile(new File(logDir, "jobmanager.out").toPath()); + if (!config.contains(WebOptions.LOG_PATH) || !config.containsKey(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY)) { + logDir = File.createTempFile("TestBaseUtils-logdir", null); + Assert.assertTrue("Unable to delete temp file", logDir.delete()); + Assert.assertTrue("Unable to create temp directory", logDir.mkdir()); + Path logFile = Files.createFile(new File(logDir, "jobmanager.log").toPath()); + Files.createFile(new File(logDir, "jobmanager.out").toPath()); + + if (!config.contains(WebOptions.LOG_PATH)) { + config.setString(WebOptions.LOG_PATH, logFile.toString()); + } + + if (!config.containsKey(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY)) { + config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, logFile.toString()); + } + } + + if (!config.contains(WebOptions.PORT)) { + config.setInteger(WebOptions.PORT, 8081); + } - config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, TASK_MANAGER_MEMORY_SIZE); - config.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); + if (!config.contains(AkkaOptions.ASK_TIMEOUT)) { + config.setString(AkkaOptions.ASK_TIMEOUT, DEFAULT_AKKA_ASK_TIMEOUT + "s"); + } - config.setString(AkkaOptions.ASK_TIMEOUT, DEFAULT_AKKA_ASK_TIMEOUT + "s"); - config.setString(AkkaOptions.STARTUP_TIMEOUT, DEFAULT_AKKA_STARTUP_TIMEOUT); + if (!config.contains(AkkaOptions.STARTUP_TIMEOUT)) { + config.setString(AkkaOptions.STARTUP_TIMEOUT, DEFAULT_AKKA_STARTUP_TIMEOUT); + } - config.setInteger(WebOptions.PORT, 8081); - config.setString(WebOptions.LOG_PATH, logFile.toString()); + if (!config.contains(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE)) { + config.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); + } - config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, logFile.toString()); + if (!config.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE)) { + config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, TASK_MANAGER_MEMORY_SIZE); + } LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(config, singleActorSystem); From 201944149d48f60282653fdb35bbdd98037f5969 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Mon, 5 Mar 2018 13:55:04 +0100 Subject: [PATCH 094/268] [FLINK-8860][flip6] stop SlotManager spamming logs for every TM heartbeat at log level 'info' This closes #5637. --- .../flink/runtime/resourcemanager/slotmanager/SlotManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index ca3371945c8fd..43e670eb9a42b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -384,7 +384,7 @@ public boolean unregisterTaskManager(InstanceID instanceId) { public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) { checkInit(); - LOG.info("Received slot report from instance {}.", instanceId); + LOG.trace("Received slot report from instance {}.", instanceId); TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId); From 6da942a3fa6c8b3750fbb13cea089aeb5870b13d Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 21 Feb 2018 15:30:16 +0100 Subject: [PATCH 095/268] [FLINK-8729][streaming] Refactor JSONGenerator to use jackson This closes #5554. --- flink-streaming-java/pom.xml | 6 --- .../streaming/api/graph/JSONGenerator.java | 54 +++++++++---------- 2 files changed, 27 insertions(+), 33 deletions(-) diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml index ef8f1622b4b08..4338fcd1576d7 100644 --- a/flink-streaming-java/pom.xml +++ b/flink-streaming-java/pom.xml @@ -67,12 +67,6 @@ under the License. 3.5 - - org.apache.sling - org.apache.sling.commons.json - 2.0.6 - - diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java index a9bb0b696f1e7..263e0aabdf197 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/JSONGenerator.java @@ -20,9 +20,9 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.sling.commons.json.JSONArray; -import org.apache.sling.commons.json.JSONException; -import org.apache.sling.commons.json.JSONObject; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; import java.util.ArrayList; import java.util.Collections; @@ -48,14 +48,15 @@ public class JSONGenerator { public static final String PARALLELISM = "parallelism"; private StreamGraph streamGraph; + private final ObjectMapper mapper = new ObjectMapper(); public JSONGenerator(StreamGraph streamGraph) { this.streamGraph = streamGraph; } - public String getJSON() throws JSONException { - JSONObject json = new JSONObject(); - JSONArray nodes = new JSONArray(); + public String getJSON() { + ObjectNode json = mapper.createObjectNode(); + ArrayNode nodes = mapper.createArrayNode(); json.put("nodes", nodes); List operatorIDs = new ArrayList(streamGraph.getVertexIDs()); Collections.sort(operatorIDs, new Comparator() { @@ -75,8 +76,8 @@ public int compare(Integer o1, Integer o2) { return json.toString(); } - private void visit(JSONArray jsonArray, List toVisit, - Map edgeRemapings) throws JSONException { + private void visit(ArrayNode jsonArray, List toVisit, + Map edgeRemapings) { Integer vertexID = toVisit.get(0); StreamNode vertex = streamGraph.getStreamNode(vertexID); @@ -84,11 +85,11 @@ private void visit(JSONArray jsonArray, List toVisit, if (streamGraph.getSourceIDs().contains(vertexID) || Collections.disjoint(vertex.getInEdges(), toVisit)) { - JSONObject node = new JSONObject(); + ObjectNode node = mapper.createObjectNode(); decorateNode(vertexID, node); if (!streamGraph.getSourceIDs().contains(vertexID)) { - JSONArray inputs = new JSONArray(); + ArrayNode inputs = mapper.createArrayNode(); node.put(PREDECESSORS, inputs); for (StreamEdge inEdge : vertex.getInEdges()) { @@ -99,7 +100,7 @@ private void visit(JSONArray jsonArray, List toVisit, decorateEdge(inputs, inEdge, mappedID); } } - jsonArray.put(node); + jsonArray.add(node); toVisit.remove(vertexID); } else { Integer iterationHead = -1; @@ -111,18 +112,18 @@ private void visit(JSONArray jsonArray, List toVisit, } } - JSONObject obj = new JSONObject(); - JSONArray iterationSteps = new JSONArray(); + ObjectNode obj = mapper.createObjectNode(); + ArrayNode iterationSteps = mapper.createArrayNode(); obj.put(STEPS, iterationSteps); obj.put(ID, iterationHead); obj.put(PACT, "IterativeDataStream"); obj.put(PARALLELISM, streamGraph.getStreamNode(iterationHead).getParallelism()); obj.put(CONTENTS, "Stream Iteration"); - JSONArray iterationInputs = new JSONArray(); + ArrayNode iterationInputs = mapper.createArrayNode(); obj.put(PREDECESSORS, iterationInputs); toVisit.remove(iterationHead); visitIteration(iterationSteps, toVisit, iterationHead, edgeRemapings, iterationInputs); - jsonArray.put(obj); + jsonArray.add(obj); } if (!toVisit.isEmpty()) { @@ -130,8 +131,8 @@ private void visit(JSONArray jsonArray, List toVisit, } } - private void visitIteration(JSONArray jsonArray, List toVisit, int headId, - Map edgeRemapings, JSONArray iterationInEdges) throws JSONException { + private void visitIteration(ArrayNode jsonArray, List toVisit, int headId, + Map edgeRemapings, ArrayNode iterationInEdges) { Integer vertexID = toVisit.get(0); StreamNode vertex = streamGraph.getStreamNode(vertexID); @@ -139,10 +140,10 @@ private void visitIteration(JSONArray jsonArray, List toVisit, int head // Ignoring head and tail to avoid redundancy if (!streamGraph.vertexIDtoLoopTimeout.containsKey(vertexID)) { - JSONObject obj = new JSONObject(); - jsonArray.put(obj); + ObjectNode obj = mapper.createObjectNode(); + jsonArray.add(obj); decorateNode(vertexID, obj); - JSONArray inEdges = new JSONArray(); + ArrayNode inEdges = mapper.createArrayNode(); obj.put(PREDECESSORS, inEdges); for (StreamEdge inEdge : vertex.getInEdges()) { @@ -161,16 +162,15 @@ private void visitIteration(JSONArray jsonArray, List toVisit, int head } - private void decorateEdge(JSONArray inputArray, StreamEdge inEdge, int mappedInputID) - throws JSONException { - JSONObject input = new JSONObject(); - inputArray.put(input); + private void decorateEdge(ArrayNode inputArray, StreamEdge inEdge, int mappedInputID) { + ObjectNode input = mapper.createObjectNode(); + inputArray.add(input); input.put(ID, mappedInputID); - input.put(SHIP_STRATEGY, inEdge.getPartitioner()); - input.put(SIDE, (inputArray.length() == 0) ? "first" : "second"); + input.put(SHIP_STRATEGY, inEdge.getPartitioner().toString()); + input.put(SIDE, (inputArray.size() == 0) ? "first" : "second"); } - private void decorateNode(Integer vertexID, JSONObject node) throws JSONException { + private void decorateNode(Integer vertexID, ObjectNode node) { StreamNode vertex = streamGraph.getStreamNode(vertexID); From eb34dd1c8e9d647137363c5b6e5be639d024be30 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 9 Mar 2018 12:03:41 +0100 Subject: [PATCH 096/268] [FLINK-8860] Change slot-report message to DEBUG --- .../flink/runtime/resourcemanager/slotmanager/SlotManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 43e670eb9a42b..120e1aa1a5ac3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -384,7 +384,7 @@ public boolean unregisterTaskManager(InstanceID instanceId) { public boolean reportSlotStatus(InstanceID instanceId, SlotReport slotReport) { checkInit(); - LOG.trace("Received slot report from instance {}.", instanceId); + LOG.debug("Received slot report from instance {}.", instanceId); TaskManagerRegistration taskManagerRegistration = taskManagerRegistrations.get(instanceId); From 2778994b68332f72bc20835e82919c1f25f1cfc3 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 8 Mar 2018 12:22:32 +0100 Subject: [PATCH 097/268] [FLINK-8896] [kafka08] remove all cancel MARKERs before trying to find partition leaders This guards us against #cancel() being called multiple times and then trying to look up an invalid topic/partition pair. --- .../streaming/connectors/kafka/internals/Kafka08Fetcher.java | 3 ++- .../streaming/connectors/kafka/FlinkKafkaConsumerBase.java | 5 ++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java index a2edb72d2dac7..96540412f0cec 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/Kafka08Fetcher.java @@ -189,7 +189,8 @@ public void runFetchLoop() throws Exception { // special marker into the queue List> partitionsToAssign = unassignedPartitionsQueue.getBatchBlocking(5000); - partitionsToAssign.remove(MARKER); + // note: if there are more markers, remove them all + partitionsToAssign.removeIf(MARKER::equals); if (!partitionsToAssign.isEmpty()) { LOG.info("Assigning {} partitions to broker threads", partitionsToAssign.size()); diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java index e19772a1f801c..82ac2c37b8faa 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java @@ -709,7 +709,10 @@ public void run() { discoveryLoopErrorRef.set(e); } finally { // calling cancel will also let the fetcher loop escape - cancel(); + // (if not running, cancel() was already called) + if (running) { + cancel(); + } } } }); From e8a0f1c0425a6d1f1397c1e116a90fe99d6cd08e Mon Sep 17 00:00:00 2001 From: Andreas Fink Date: Mon, 5 Mar 2018 18:26:57 +0100 Subject: [PATCH 098/268] [FLINK-8091] [scripts] Support running historyserver in foreground --- docs/monitoring/historyserver.md | 2 +- flink-dist/src/main/flink-bin/bin/flink-console.sh | 6 +++++- flink-dist/src/main/flink-bin/bin/historyserver.sh | 10 +++++++--- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/docs/monitoring/historyserver.md b/docs/monitoring/historyserver.md index dfbbc8edbc1d4..9c68e6525083d 100644 --- a/docs/monitoring/historyserver.md +++ b/docs/monitoring/historyserver.md @@ -37,7 +37,7 @@ After you have configured the HistoryServer *and* JobManager, you start and stop ```sh # Start or stop the HistoryServer -bin/historyserver.sh (start|stop) +bin/historyserver.sh (start|start-foreground|stop) ``` By default, this server binds to `localhost` and listens at port `8082`. diff --git a/flink-dist/src/main/flink-bin/bin/flink-console.sh b/flink-dist/src/main/flink-bin/bin/flink-console.sh index 574376c6ede6d..3ccbbd0b99d5b 100644 --- a/flink-dist/src/main/flink-bin/bin/flink-console.sh +++ b/flink-dist/src/main/flink-bin/bin/flink-console.sh @@ -19,7 +19,7 @@ # Start a Flink service as a console application. Must be stopped with Ctrl-C # or with SIGTERM by kill or the controlling process. -USAGE="Usage: flink-console.sh (jobmanager|taskmanager|zookeeper) [args]" +USAGE="Usage: flink-console.sh (jobmanager|taskmanager|historyserver|zookeeper) [args]" SERVICE=$1 ARGS=("${@:2}") # get remaining arguments as array @@ -42,6 +42,10 @@ case $SERVICE in CLASS_TO_RUN=org.apache.flink.runtime.taskexecutor.TaskManagerRunner ;; + (historyserver) + CLASS_TO_RUN=org.apache.flink.runtime.webmonitor.history.HistoryServer + ;; + (zookeeper) CLASS_TO_RUN=org.apache.flink.runtime.zookeeper.FlinkZooKeeperQuorumPeer ;; diff --git a/flink-dist/src/main/flink-bin/bin/historyserver.sh b/flink-dist/src/main/flink-bin/bin/historyserver.sh index adc966004cc35..150ab9469aeed 100644 --- a/flink-dist/src/main/flink-bin/bin/historyserver.sh +++ b/flink-dist/src/main/flink-bin/bin/historyserver.sh @@ -18,7 +18,7 @@ ################################################################################ # Start/stop a Flink HistoryServer -USAGE="Usage: historyserver.sh (start|stop)" +USAGE="Usage: historyserver.sh (start|start-foreground|stop)" STARTSTOP=$1 @@ -27,8 +27,12 @@ bin=`cd "$bin"; pwd` . "$bin"/config.sh -if [[ $STARTSTOP == "start" ]]; then +if [[ $STARTSTOP == "start" ]] || [[ $STARTSTOP == "start-foreground" ]]; then args=("--configDir" "${FLINK_CONF_DIR}") fi -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP historyserver "${args[@]}" +if [[ $STARTSTOP == "start-foreground" ]]; then + exec "${FLINK_BIN_DIR}"/flink-console.sh historyserver "${args[@]}" +else + "${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP historyserver "${args[@]}" +fi From a75815aae4362dee0470345e0209096e503dc1ef Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 8 Mar 2018 11:51:38 +0100 Subject: [PATCH 099/268] [FLINK-8854] [table] Fix schema mapping with time attributes This closes #5662. --- .../KafkaJsonTableSourceFactoryTestBase.java | 20 +++++-- .../table/descriptors/SchemaValidator.scala | 19 +++++- .../table/sources/definedTimeAttributes.scala | 19 +++++- .../sources/tsextractors/ExistingField.scala | 10 +++- .../BoundedOutOfOrderTimestamps.scala | 10 ++++ .../descriptors/SchemaValidatorTest.scala | 58 ++++++++++++++++++- 6 files changed, 120 insertions(+), 16 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java index 2b081a9f9157d..583b71dd4e88a 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSourceFactoryTestBase.java @@ -25,10 +25,13 @@ import org.apache.flink.table.descriptors.FormatDescriptor; import org.apache.flink.table.descriptors.Json; import org.apache.flink.table.descriptors.Kafka; +import org.apache.flink.table.descriptors.Rowtime; import org.apache.flink.table.descriptors.Schema; import org.apache.flink.table.descriptors.TestTableSourceDescriptor; import org.apache.flink.table.sources.TableSource; import org.apache.flink.table.sources.TableSourceFactoryService; +import org.apache.flink.table.sources.tsextractors.ExistingField; +import org.apache.flink.table.sources.wmstrategies.PreserveWatermarks; import org.junit.Test; @@ -55,9 +58,11 @@ public abstract class KafkaJsonTableSourceFactoryTestBase { " 'type': 'integer'" + " }," + " 'time': {" + - " 'description': 'Age in years'," + - " 'type': 'number'" + - " }" + " }," + + " 'description': 'row time'," + + " 'type': 'string'," + + " 'format': 'date-time'" + + " }" + + " }," + " 'required': ['name', 'count', 'time']" + "}"; @@ -89,9 +94,10 @@ private void testTableSource(FormatDescriptor format) { // construct table source using a builder final Map tableJsonMapping = new HashMap<>(); + tableJsonMapping.put("name", "name"); tableJsonMapping.put("fruit-name", "name"); tableJsonMapping.put("count", "count"); - tableJsonMapping.put("event-time", "time"); + tableJsonMapping.put("time", "time"); final Properties props = new Properties(); props.put("group.id", "test-group"); @@ -112,10 +118,11 @@ private void testTableSource(FormatDescriptor format) { TableSchema.builder() .field("fruit-name", Types.STRING) .field("count", Types.BIG_INT) - .field("event-time", Types.BIG_DEC) + .field("event-time", Types.SQL_TIMESTAMP) .field("proc-time", Types.SQL_TIMESTAMP) .build()) .withProctimeAttribute("proc-time") + .withRowtimeAttribute("event-time", new ExistingField("time"), PreserveWatermarks.INSTANCE()) .build(); // construct table source using descriptors and table source factory @@ -135,7 +142,8 @@ private void testTableSource(FormatDescriptor format) { new Schema() .field("fruit-name", Types.STRING).from("name") .field("count", Types.BIG_INT) // no from so it must match with the input - .field("event-time", Types.BIG_DEC).from("time") + .field("event-time", Types.SQL_TIMESTAMP).rowtime( + new Rowtime().timestampsFromField("time").watermarksFromSource()) .field("proc-time", Types.SQL_TIMESTAMP).proctime()); final TableSource factorySource = TableSourceFactoryService.findAndCreateTableSource(testDesc); diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala index 0a2391175bf01..9cb3258d68aba 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/SchemaValidator.scala @@ -23,7 +23,7 @@ import java.util.Optional import org.apache.flink.table.api.{TableSchema, ValidationException} import org.apache.flink.table.descriptors.DescriptorProperties.{toJava, toScala} -import org.apache.flink.table.descriptors.RowtimeValidator.{ROWTIME, ROWTIME_TIMESTAMPS_TYPE} +import org.apache.flink.table.descriptors.RowtimeValidator.{ROWTIME, ROWTIME_TIMESTAMPS_FROM, ROWTIME_TIMESTAMPS_TYPE, ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD} import org.apache.flink.table.descriptors.SchemaValidator._ import org.apache.flink.table.sources.RowtimeAttributeDescriptor @@ -148,6 +148,13 @@ object SchemaValidator { val schema = properties.getTableSchema(SCHEMA) + // add all source fields first because rowtime might reference one of them + toScala(sourceSchema).map(_.getColumnNames).foreach { names => + names.foreach { name => + mapping.put(name, name) + } + } + // add all schema fields first for implicit mappings schema.getColumnNames.foreach { name => mapping.put(name, name) @@ -198,14 +205,20 @@ object SchemaValidator { val isProctime = properties .getOptionalBoolean(s"$SCHEMA.$i.$SCHEMA_PROCTIME") .orElse(false) - val isRowtime = properties - .containsKey(s"$SCHEMA.$i.$ROWTIME_TIMESTAMPS_TYPE") + val tsType = s"$SCHEMA.$i.$ROWTIME_TIMESTAMPS_TYPE" + val isRowtime = properties.containsKey(tsType) if (!isProctime && !isRowtime) { // check for a aliasing val fieldName = properties.getOptionalString(s"$SCHEMA.$i.$SCHEMA_FROM") .orElse(n) builder.field(fieldName, t) } + // only use the rowtime attribute if it references a field + else if (isRowtime && + properties.getString(tsType) == ROWTIME_TIMESTAMPS_TYPE_VALUE_FROM_FIELD) { + val field = properties.getString(s"$SCHEMA.$i.$ROWTIME_TIMESTAMPS_FROM") + builder.field(field, t) + } } builder.build() diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala index f09baa35b73f1..73b76a58584eb 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/definedTimeAttributes.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.sources import java.util +import java.util.Objects import org.apache.flink.table.api.TableSchema import org.apache.flink.table.api.Types @@ -65,9 +66,9 @@ trait DefinedRowtimeAttributes { * @param watermarkStrategy The watermark strategy associated with the attribute. */ class RowtimeAttributeDescriptor( - attributeName: String, - timestampExtractor: TimestampExtractor, - watermarkStrategy: WatermarkStrategy) { + val attributeName: String, + val timestampExtractor: TimestampExtractor, + val watermarkStrategy: WatermarkStrategy) { /** Returns the name of the rowtime attribute. */ def getAttributeName: String = attributeName @@ -77,4 +78,16 @@ class RowtimeAttributeDescriptor( /** Returns the [[WatermarkStrategy]] for the attribute. */ def getWatermarkStrategy: WatermarkStrategy = watermarkStrategy + + override def equals(other: Any): Boolean = other match { + case that: RowtimeAttributeDescriptor => + Objects.equals(attributeName, that.attributeName) && + Objects.equals(timestampExtractor, that.timestampExtractor) && + Objects.equals(watermarkStrategy, that.watermarkStrategy) + case _ => false + } + + override def hashCode(): Int = { + Objects.hash(attributeName, timestampExtractor, watermarkStrategy) + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/tsextractors/ExistingField.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/tsextractors/ExistingField.scala index 12cd564395cea..866029bcd3859 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/tsextractors/ExistingField.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/tsextractors/ExistingField.scala @@ -27,7 +27,7 @@ import org.apache.flink.table.expressions.{Cast, Expression, ResolvedFieldRefere * * @param field The field to convert into a rowtime attribute. */ -class ExistingField(field: String) extends TimestampExtractor { +class ExistingField(val field: String) extends TimestampExtractor { override def getArgumentFields: Array[String] = Array(field) @@ -65,4 +65,12 @@ class ExistingField(field: String) extends TimestampExtractor { } } + override def equals(other: Any): Boolean = other match { + case that: ExistingField => field == that.field + case _ => false + } + + override def hashCode(): Int = { + field.hashCode + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/wmstrategies/BoundedOutOfOrderTimestamps.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/wmstrategies/BoundedOutOfOrderTimestamps.scala index 8f7c23560187b..4718bad57d455 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/wmstrategies/BoundedOutOfOrderTimestamps.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/wmstrategies/BoundedOutOfOrderTimestamps.scala @@ -38,4 +38,14 @@ final class BoundedOutOfOrderTimestamps(val delay: Long) extends PeriodicWaterma } override def getWatermark: Watermark = new Watermark(maxTimestamp - delay) + + override def equals(other: Any): Boolean = other match { + case that: BoundedOutOfOrderTimestamps => + delay == that.delay + case _ => false + } + + override def hashCode(): Int = { + delay.hashCode() + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala index ba05dfff2074d..bf7b84b8e769d 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/SchemaValidatorTest.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.descriptors import java.util.Optional import org.apache.flink.table.api.{TableSchema, Types} -import org.apache.flink.table.sources.tsextractors.StreamRecordTimestamp +import org.apache.flink.table.sources.tsextractors.{ExistingField, StreamRecordTimestamp} import org.apache.flink.table.sources.wmstrategies.PreserveWatermarks import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Test @@ -34,7 +34,7 @@ import scala.collection.JavaConverters._ class SchemaValidatorTest { @Test - def testSchema(): Unit = { + def testSchemaWithRowtimeFromSource(): Unit = { val desc1 = Schema() .field("otherField", Types.STRING).from("csvField") .field("abcField", Types.STRING) @@ -60,7 +60,11 @@ class SchemaValidatorTest { assertTrue(rowtime.getWatermarkStrategy.isInstanceOf[PreserveWatermarks]) // test field mapping - val expectedMapping = Map("otherField" -> "csvField", "abcField" -> "abcField").asJava + val expectedMapping = Map( + "otherField" -> "csvField", + "csvField" -> "csvField", + "abcField" -> "abcField", + "myField" -> "myField").asJava assertEquals( expectedMapping, SchemaValidator.deriveFieldMapping(props, Optional.of(inputSchema))) @@ -73,4 +77,52 @@ class SchemaValidatorTest { .build() assertEquals(expectedFormatSchema, formatSchema) } + + @Test + def testSchemaWithRowtimeFromField(): Unit = { + val desc1 = Schema() + .field("otherField", Types.STRING).from("csvField") + .field("abcField", Types.STRING) + .field("p", Types.SQL_TIMESTAMP).proctime() + .field("r", Types.SQL_TIMESTAMP).rowtime( + Rowtime().timestampsFromField("myTime").watermarksFromSource()) + val props = new DescriptorProperties() + desc1.addProperties(props) + + val inputSchema = TableSchema.builder() + .field("csvField", Types.STRING) + .field("abcField", Types.STRING) + .field("myField", Types.BOOLEAN) + .field("myTime", Types.SQL_TIMESTAMP) + .build() + + // test proctime + assertEquals(Optional.of("p"), SchemaValidator.deriveProctimeAttribute(props)) + + // test rowtime + val rowtime = SchemaValidator.deriveRowtimeAttributes(props).get(0) + assertEquals("r", rowtime.getAttributeName) + assertTrue(rowtime.getTimestampExtractor.isInstanceOf[ExistingField]) + assertTrue(rowtime.getWatermarkStrategy.isInstanceOf[PreserveWatermarks]) + + // test field mapping + val expectedMapping = Map( + "otherField" -> "csvField", + "csvField" -> "csvField", + "abcField" -> "abcField", + "myField" -> "myField", + "myTime" -> "myTime").asJava + assertEquals( + expectedMapping, + SchemaValidator.deriveFieldMapping(props, Optional.of(inputSchema))) + + // test field format + val formatSchema = SchemaValidator.deriveFormatFields(props) + val expectedFormatSchema = TableSchema.builder() + .field("csvField", Types.STRING) // aliased + .field("abcField", Types.STRING) + .field("myTime", Types.SQL_TIMESTAMP) + .build() + assertEquals(expectedFormatSchema, formatSchema) + } } From adae7d9672bf42fc44a9ceb31b0802007870af81 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 22 Feb 2018 14:11:13 +0100 Subject: [PATCH 100/268] [hotfix] [network] Rename RecordWriter#closeBufferConsumer() to closeBufferBuilder() --- .../flink/runtime/io/network/api/writer/RecordWriter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index 4ec28631e07f4..c35c7f3e8aa90 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -175,7 +175,7 @@ public void flushAll() { public void clearBuffers() { for (int targetChannel = 0; targetChannel < numChannels; targetChannel++) { RecordSerializer serializer = serializers[targetChannel]; - closeBufferConsumer(targetChannel); + closeBufferBuilder(targetChannel); serializer.clear(); } } @@ -213,7 +213,7 @@ private BufferBuilder requestNewBufferBuilder(int targetChannel) throws IOExcept return bufferBuilder; } - private void closeBufferConsumer(int targetChannel) { + private void closeBufferBuilder(int targetChannel) { if (bufferBuilders[targetChannel].isPresent()) { bufferBuilders[targetChannel].get().finish(); bufferBuilders[targetChannel] = Optional.empty(); From 7e2e25cfa6be737803a74b75ffb231568a51513e Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 22 Feb 2018 14:17:06 +0100 Subject: [PATCH 101/268] [hotfix] [network] Various minor improvements --- .../netty/CreditBasedSequenceNumberingViewReader.java | 3 ++- .../runtime/io/network/netty/PartitionRequestQueue.java | 5 ++++- .../io/network/netty/SequenceNumberingViewReader.java | 1 + .../runtime/io/network/partition/SubpartitionTestBase.java | 2 +- 4 files changed, 8 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java index 9acbbacf2735b..8fc7ef4842b48 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/CreditBasedSequenceNumberingViewReader.java @@ -114,7 +114,7 @@ public boolean isRegisteredAsAvailable() { */ @Override public boolean isAvailable() { - // BEWARE: this must be in sync with #isAvailable()! + // BEWARE: this must be in sync with #isAvailable(BufferAndBacklog)! return hasBuffersAvailable() && (numCreditsAvailable > 0 || subpartitionView.nextBufferIsEvent()); } @@ -130,6 +130,7 @@ public boolean isAvailable() { * current buffer and backlog including information about the next buffer */ private boolean isAvailable(BufferAndBacklog bufferAndBacklog) { + // BEWARE: this must be in sync with #isAvailable()! return bufferAndBacklog.isMoreAvailable() && (numCreditsAvailable > 0 || bufferAndBacklog.nextBufferIsEvent()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java index d63a88e718276..8c05b8208f90f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java @@ -38,6 +38,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.ArrayDeque; import java.util.Set; @@ -52,7 +54,7 @@ */ class PartitionRequestQueue extends ChannelInboundHandlerAdapter { - private final Logger LOG = LoggerFactory.getLogger(PartitionRequestQueue.class); + private static final Logger LOG = LoggerFactory.getLogger(PartitionRequestQueue.class); private final ChannelFutureListener writeListener = new WriteAndFlushNextMessageIfPossibleListener(); @@ -278,6 +280,7 @@ private void registerAvailableReader(NetworkSequenceViewReader reader) { reader.setRegisteredAsAvailable(true); } + @Nullable private NetworkSequenceViewReader pollAvailableReader() { NetworkSequenceViewReader reader = availableReaders.poll(); if (reader != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java index 6a83af1383782..054046f086a75 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/SequenceNumberingViewReader.java @@ -148,6 +148,7 @@ public String toString() { "requestLock=" + requestLock + ", receiverId=" + receiverId + ", sequenceNumber=" + sequenceNumber + + ", isRegisteredAsAvailable=" + isRegisteredAvailable + '}'; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index 215726b3b5ad1..a3f18f6c18cea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -134,7 +134,7 @@ private void verifyViewReleasedAfterParentRelease(ResultSubpartition partition) assertTrue(view.isReleased()); } - protected void assertNextBuffer( + static void assertNextBuffer( ResultSubpartitionView readView, int expectedReadableBufferSize, boolean expectedIsMoreAvailable, From b3a93c569a3eca42b99323491d40f5bf0ad40628 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 23 Feb 2018 10:35:41 +0100 Subject: [PATCH 102/268] [hotfix] [network] [tests] Make AwaitableBufferAvailablityListener thread-safe This is called asynchronously by the spill writer and thus may need synchronization on incrementing the counter but definately had visibility issues with the counter. Using an AtomicLong fixes that. --- .../AwaitableBufferAvailablityListener.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AwaitableBufferAvailablityListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AwaitableBufferAvailablityListener.java index 2b6b834c0b560..6cf9d64f1b3bc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AwaitableBufferAvailablityListener.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/AwaitableBufferAvailablityListener.java @@ -18,29 +18,31 @@ package org.apache.flink.runtime.io.network.partition; +import java.util.concurrent.atomic.AtomicLong; + /** * Test implementation of {@link BufferAvailabilityListener}. */ class AwaitableBufferAvailablityListener implements BufferAvailabilityListener { - private long numNotifications; + private final AtomicLong numNotifications = new AtomicLong(); @Override public void notifyDataAvailable() { - ++numNotifications; + numNotifications.getAndIncrement(); } public long getNumNotifications() { - return numNotifications; + return numNotifications.get(); } public void resetNotificationCounters() { - numNotifications = 0; + numNotifications.set(0L); } void awaitNotifications(long awaitedNumNotifications, long timeoutMillis) throws InterruptedException { long deadline = System.currentTimeMillis() + timeoutMillis; - while (numNotifications < awaitedNumNotifications && System.currentTimeMillis() < deadline) { + while (numNotifications.get() < awaitedNumNotifications && System.currentTimeMillis() < deadline) { Thread.sleep(1); } } From 5c7457aa2aece89b77e9a9402cc514f90e083a69 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 23 Feb 2018 10:19:58 +0100 Subject: [PATCH 103/268] [FLINK-8755] [network] Fix SpilledSubpartitionView relying on the backlog for determining whether more data is available Fix SpilledSubpartitionView#getNextBuffer() to not only rely on the backlog: instead it is sufficient to also return true if the next buffer is an event since either there is a real buffer enqueued (reflected by the backlog) or at least one event. --- .../runtime/io/network/partition/SpilledSubpartitionView.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java index 378b0867d6fee..2a6a71f05d667 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java @@ -148,7 +148,7 @@ public BufferAndBacklog getNextBuffer() throws IOException, InterruptedException } int newBacklog = parent.decreaseBuffersInBacklog(current); - return new BufferAndBacklog(current, newBacklog > 0, newBacklog, nextBufferIsEvent); + return new BufferAndBacklog(current, newBacklog > 0 || nextBufferIsEvent, newBacklog, nextBufferIsEvent); } @Nullable From 835adcc373ce169f202055e9b4f9dc3fb9123772 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Mon, 26 Feb 2018 16:27:44 +0100 Subject: [PATCH 104/268] [FLINK-8786] [network] Fix SpillableSubpartitionView#getNextBuffer returning wrong isMoreAvailable when processing last in-memory buffer When processing the last in-memory buffer in SpillableSubpartitionView#getNextBuffer while the rest of the buffers are spilled, need to rely on the spilled view's isAvailable instead of always setting the isMoreAvailable flag of the returned BufferAndBacklog to false. --- .../runtime/io/network/partition/SpillableSubpartitionView.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java index 3c73e43d8cb9f..0f51bc8b03e37 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java @@ -177,7 +177,7 @@ public BufferAndBacklog getNextBuffer() throws IOException, InterruptedException SpilledSubpartitionView spilled = spilledView; if (spilled != null) { if (current != null) { - return new BufferAndBacklog(current, isMoreAvailable, newBacklog, spilled.nextBufferIsEvent()); + return new BufferAndBacklog(current, spilled.isAvailable(), newBacklog, spilled.nextBufferIsEvent()); } else { return spilled.getNextBuffer(); } From d1a969f7ad018ef44f40f974eb49ba004494fcdf Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 23 Feb 2018 12:13:20 +0100 Subject: [PATCH 105/268] [FLINK-8755] [FLINK-8786] [network] Add and improve subpartition tests + also improve the subpartition tests in general to reduce some duplication This closes #5581 --- .../partition/SpillableSubpartitionView.java | 2 +- .../partition/PipelinedSubpartitionTest.java | 11 +- .../partition/SpillableSubpartitionTest.java | 130 ++++++------------ .../partition/SubpartitionTestBase.java | 78 ++++++++++- 4 files changed, 121 insertions(+), 100 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java index 0f51bc8b03e37..65790d79df28d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java @@ -167,7 +167,7 @@ public BufferAndBacklog getNextBuffer() throws IOException, InterruptedException parent.updateStatistics(current); // if we are spilled (but still process a non-spilled nextBuffer), we don't know the - // state of nextBufferIsEvent... + // state of nextBufferIsEvent or whether more buffers are available if (spilledView == null) { return new BufferAndBacklog(current, isMoreAvailable, newBacklog, nextBufferIsEvent); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index ee678abc4ccc4..bc66c9d292dc7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -135,7 +135,8 @@ public void testAddNonEmptyNotFinishedBuffer() throws Exception { bufferBuilder.appendAndCommit(ByteBuffer.allocate(1024)); subpartition.add(bufferBuilder.createBufferConsumer()); - assertNextBuffer(readView, 1024, false, 1); + // note that since the buffer builder is not finished, there is still a retained instance! + assertNextBuffer(readView, 1024, false, 1, false, false); assertEquals(1, subpartition.getBuffersInBacklog()); } finally { readView.releaseAllResources(); @@ -157,7 +158,7 @@ public void testUnfinishedBufferBehindFinished() throws Exception { subpartition.add(createFilledBufferConsumer(1025)); // finished subpartition.add(createFilledBufferBuilder(1024).createBufferConsumer()); // not finished - assertNextBuffer(readView, 1025, false, 1); + assertNextBuffer(readView, 1025, false, 1, false, true); } finally { subpartition.release(); } @@ -178,8 +179,8 @@ public void testFlushWithUnfinishedBufferBehindFinished() throws Exception { subpartition.add(createFilledBufferBuilder(1024).createBufferConsumer()); // not finished subpartition.flush(); - assertNextBuffer(readView, 1025, true, 1); - assertNextBuffer(readView, 1024, false, 1); + assertNextBuffer(readView, 1025, true, 1, false, true); + assertNextBuffer(readView, 1024, false, 1, false, false); } finally { subpartition.release(); } @@ -208,7 +209,7 @@ public void testMultipleEmptyBuffers() throws Exception { subpartition.add(createFilledBufferConsumer(1024)); assertEquals(2, availablityListener.getNumNotifications()); - assertNextBuffer(readView, 1024, false, 0); + assertNextBuffer(readView, 1024, false, 0, false, true); } finally { readView.releaseAllResources(); subpartition.release(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java index e41a85c5207b4..840669e7c3fcc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java @@ -24,13 +24,13 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsyncWithNoOpBufferFileWriter; +import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; -import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; -import org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog; import org.junit.AfterClass; import org.junit.Assert; @@ -52,7 +52,6 @@ import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; @@ -190,10 +189,13 @@ public void testConsumeSpilledPartition() throws Exception { SpillableSubpartition partition = createSubpartition(); BufferConsumer bufferConsumer = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE); + BufferConsumer eventBufferConsumer = + EventSerializer.toBufferConsumer(new CancelCheckpointMarker(1)); + final int eventSize = eventBufferConsumer.getWrittenBytes(); partition.add(bufferConsumer.copy()); partition.add(bufferConsumer.copy()); - partition.add(BufferBuilderTestUtils.createEventBufferConsumer(BUFFER_DATA_SIZE)); + partition.add(eventBufferConsumer); partition.add(bufferConsumer); assertEquals(4, partition.getTotalNumberOfBuffers()); @@ -207,13 +209,13 @@ public void testConsumeSpilledPartition() throws Exception { // still same statistics assertEquals(4, partition.getTotalNumberOfBuffers()); assertEquals(3, partition.getBuffersInBacklog()); - assertEquals(BUFFER_DATA_SIZE * 4, partition.getTotalNumberOfBytes()); + assertEquals(BUFFER_DATA_SIZE * 3 + eventSize, partition.getTotalNumberOfBytes()); partition.finish(); // + one EndOfPartitionEvent assertEquals(5, partition.getTotalNumberOfBuffers()); assertEquals(3, partition.getBuffersInBacklog()); - assertEquals(BUFFER_DATA_SIZE * 4 + 4, partition.getTotalNumberOfBytes()); + assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); AwaitableBufferAvailablityListener listener = new AwaitableBufferAvailablityListener(); SpilledSubpartitionView reader = (SpilledSubpartitionView) partition.createReadView(listener); @@ -221,59 +223,24 @@ public void testConsumeSpilledPartition() throws Exception { assertEquals(1, listener.getNumNotifications()); assertFalse(reader.nextBufferIsEvent()); // buffer - BufferAndBacklog read = reader.getNextBuffer(); - assertNotNull(read); - assertTrue(read.buffer().isBuffer()); + assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 2, false, true); assertEquals(2, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - assertFalse(read.buffer().isRecycled()); - read.buffer().recycleBuffer(); - assertTrue(read.buffer().isRecycled()); - assertFalse(read.nextBufferIsEvent()); assertFalse(reader.nextBufferIsEvent()); // buffer - read = reader.getNextBuffer(); - assertNotNull(read); - assertTrue(read.buffer().isBuffer()); + assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 1, true, true); assertEquals(1, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - assertFalse(read.buffer().isRecycled()); - read.buffer().recycleBuffer(); - assertTrue(read.buffer().isRecycled()); - assertTrue(read.nextBufferIsEvent()); assertTrue(reader.nextBufferIsEvent()); // event - read = reader.getNextBuffer(); - assertNotNull(read); - assertFalse(read.buffer().isBuffer()); + assertNextEvent(reader, eventSize, CancelCheckpointMarker.class, true, 1, false, true); assertEquals(1, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - read.buffer().recycleBuffer(); - assertFalse(read.nextBufferIsEvent()); assertFalse(reader.nextBufferIsEvent()); // buffer - read = reader.getNextBuffer(); - assertNotNull(read); - assertTrue(read.buffer().isBuffer()); + assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 0, true, true); assertEquals(0, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - assertFalse(read.buffer().isRecycled()); - read.buffer().recycleBuffer(); - assertTrue(read.buffer().isRecycled()); - assertTrue(read.nextBufferIsEvent()); assertTrue(reader.nextBufferIsEvent()); // end of partition event - read = reader.getNextBuffer(); - assertNotNull(read); - assertFalse(read.buffer().isBuffer()); + assertNextEvent(reader, 4, EndOfPartitionEvent.class, false, 0, false, true); assertEquals(0, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - assertEquals(EndOfPartitionEvent.class, - EventSerializer.fromBuffer(read.buffer(), ClassLoader.getSystemClassLoader()).getClass()); - assertFalse(read.buffer().isRecycled()); - read.buffer().recycleBuffer(); - assertTrue(read.buffer().isRecycled()); - assertFalse(read.nextBufferIsEvent()); // finally check that the bufferConsumer has been freed after a successful (or failed) write final long deadline = System.currentTimeMillis() + 30_000L; // 30 secs @@ -292,10 +259,13 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception SpillableSubpartition partition = createSubpartition(); BufferConsumer bufferConsumer = createFilledBufferConsumer(BUFFER_DATA_SIZE, BUFFER_DATA_SIZE); + BufferConsumer eventBufferConsumer = + EventSerializer.toBufferConsumer(new CancelCheckpointMarker(1)); + final int eventSize = eventBufferConsumer.getWrittenBytes(); partition.add(bufferConsumer.copy()); partition.add(bufferConsumer.copy()); - partition.add(BufferBuilderTestUtils.createEventBufferConsumer(BUFFER_DATA_SIZE)); + partition.add(eventBufferConsumer); partition.add(bufferConsumer); partition.finish(); @@ -311,17 +281,12 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception assertFalse(bufferConsumer.isRecycled()); assertFalse(reader.nextBufferIsEvent()); - BufferAndBacklog read = reader.getNextBuffer(); // first buffer (non-spilled) - assertNotNull(read); - assertTrue(read.buffer().isBuffer()); + // first buffer (non-spilled) + assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 2, false, false); assertEquals(BUFFER_DATA_SIZE, partition.getTotalNumberOfBytes()); // only updated when getting/spilling the buffers assertEquals(2, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - read.buffer().recycleBuffer(); - assertTrue(read.isMoreAvailable()); assertEquals(1, listener.getNumNotifications()); // since isMoreAvailable is set to true, no need for notification assertFalse(bufferConsumer.isRecycled()); - assertFalse(read.nextBufferIsEvent()); // Spill now assertEquals(3, partition.releaseMemory()); @@ -330,59 +295,44 @@ public void testConsumeSpillablePartitionSpilledDuringConsume() throws Exception assertEquals(5, partition.getTotalNumberOfBuffers()); assertEquals(2, partition.getBuffersInBacklog()); // only updated when getting/spilling the buffers but without the nextBuffer (kept in memory) - assertEquals(BUFFER_DATA_SIZE * 3 + 4, partition.getTotalNumberOfBytes()); + assertEquals(BUFFER_DATA_SIZE * 2 + eventSize + 4, partition.getTotalNumberOfBytes()); + // wait for successfully spilling all buffers (before that we may not access any spilled buffer and cannot rely on isMoreAvailable!) listener.awaitNotifications(2, 30_000); // Spiller finished assertEquals(2, listener.getNumNotifications()); + // after consuming and releasing the next buffer, the bufferConsumer may be freed, + // depending on the timing of the last write operation + // -> retain once so that we can check below + Buffer buffer = bufferConsumer.build(); + buffer.retainBuffer(); + assertFalse(reader.nextBufferIsEvent()); // second buffer (retained in SpillableSubpartition#nextBuffer) - read = reader.getNextBuffer(); - assertNotNull(read); - assertTrue(read.buffer().isBuffer()); - assertEquals(BUFFER_DATA_SIZE * 4 + 4, partition.getTotalNumberOfBytes()); // finally integrates the nextBuffer statistics + assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 1, true, false); + assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); // finally integrates the nextBuffer statistics assertEquals(1, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - read.buffer().recycleBuffer(); - // now the bufferConsumer may be freed, depending on the timing of the write operation - // -> let's do this check at the end of the test (to save some time) - assertTrue(read.nextBufferIsEvent()); + + bufferConsumer.close(); // recycle the retained buffer from above (should be the last reference!) assertTrue(reader.nextBufferIsEvent()); // the event (spilled) - read = reader.getNextBuffer(); - assertNotNull(read); - assertFalse(read.buffer().isBuffer()); - assertEquals(BUFFER_DATA_SIZE * 4 + 4, partition.getTotalNumberOfBytes()); // already updated during spilling + assertNextEvent(reader, eventSize, CancelCheckpointMarker.class, true, 1, false, true); + assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); // already updated during spilling assertEquals(1, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - read.buffer().recycleBuffer(); - assertFalse(read.nextBufferIsEvent()); assertFalse(reader.nextBufferIsEvent()); // last buffer (spilled) - read = reader.getNextBuffer(); - assertNotNull(read); - assertTrue(read.buffer().isBuffer()); - assertEquals(BUFFER_DATA_SIZE * 4 + 4, partition.getTotalNumberOfBytes()); // already updated during spilling + assertNextBuffer(reader, BUFFER_DATA_SIZE, true, 0, true, true); + assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); // already updated during spilling assertEquals(0, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - assertFalse(read.buffer().isRecycled()); - read.buffer().recycleBuffer(); - assertTrue(read.buffer().isRecycled()); - assertTrue(read.nextBufferIsEvent()); + + buffer.recycleBuffer(); + assertTrue(buffer.isRecycled()); // End of partition assertTrue(reader.nextBufferIsEvent()); - read = reader.getNextBuffer(); - assertNotNull(read); - assertEquals(BUFFER_DATA_SIZE * 4 + 4, partition.getTotalNumberOfBytes()); // already updated during spilling + assertNextEvent(reader, 4, EndOfPartitionEvent.class, false, 0, false, true); + assertEquals(BUFFER_DATA_SIZE * 3 + eventSize + 4, partition.getTotalNumberOfBytes()); // already updated during spilling assertEquals(0, partition.getBuffersInBacklog()); - assertEquals(partition.getBuffersInBacklog(), read.buffersInBacklog()); - assertEquals(EndOfPartitionEvent.class, - EventSerializer.fromBuffer(read.buffer(), ClassLoader.getSystemClassLoader()).getClass()); - assertFalse(read.buffer().isRecycled()); - read.buffer().recycleBuffer(); - assertTrue(read.buffer().isRecycled()); - assertFalse(read.nextBufferIsEvent()); // finally check that the bufferConsumer has been freed after a successful (or failed) write final long deadline = System.currentTimeMillis() + 30_000L; // 30 secs diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java index a3f18f6c18cea..8c902157da977 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SubpartitionTestBase.java @@ -18,19 +18,26 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.event.AbstractEvent; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; import org.apache.flink.runtime.io.network.buffer.BufferConsumer; import org.apache.flink.util.TestLogger; import org.junit.Test; +import javax.annotation.Nullable; + import java.io.IOException; import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; @@ -138,11 +145,74 @@ static void assertNextBuffer( ResultSubpartitionView readView, int expectedReadableBufferSize, boolean expectedIsMoreAvailable, - int expectedBuffersInBacklog) throws IOException, InterruptedException { + int expectedBuffersInBacklog, + boolean expectedNextBufferIsEvent, + boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException { + assertNextBufferOrEvent( + readView, + expectedReadableBufferSize, + true, + null, + expectedIsMoreAvailable, + expectedBuffersInBacklog, + expectedNextBufferIsEvent, + expectedRecycledAfterRecycle); + } + + static void assertNextEvent( + ResultSubpartitionView readView, + int expectedReadableBufferSize, + Class expectedEventClass, + boolean expectedIsMoreAvailable, + int expectedBuffersInBacklog, + boolean expectedNextBufferIsEvent, + boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException { + assertNextBufferOrEvent( + readView, + expectedReadableBufferSize, + false, + expectedEventClass, + expectedIsMoreAvailable, + expectedBuffersInBacklog, + expectedNextBufferIsEvent, + expectedRecycledAfterRecycle); + } + + private static void assertNextBufferOrEvent( + ResultSubpartitionView readView, + int expectedReadableBufferSize, + boolean expectedIsBuffer, + @Nullable Class expectedEventClass, + boolean expectedIsMoreAvailable, + int expectedBuffersInBacklog, + boolean expectedNextBufferIsEvent, + boolean expectedRecycledAfterRecycle) throws IOException, InterruptedException { + checkArgument(expectedEventClass == null || !expectedIsBuffer); + ResultSubpartition.BufferAndBacklog bufferAndBacklog = readView.getNextBuffer(); - assertEquals(expectedReadableBufferSize, bufferAndBacklog.buffer().readableBytes()); - assertEquals(expectedIsMoreAvailable, bufferAndBacklog.isMoreAvailable()); - assertEquals(expectedBuffersInBacklog, bufferAndBacklog.buffersInBacklog()); + assertNotNull(bufferAndBacklog); + try { + assertEquals("buffer size", expectedReadableBufferSize, + bufferAndBacklog.buffer().readableBytes()); + assertEquals("buffer or event", expectedIsBuffer, + bufferAndBacklog.buffer().isBuffer()); + if (expectedEventClass != null) { + assertThat(EventSerializer + .fromBuffer(bufferAndBacklog.buffer(), ClassLoader.getSystemClassLoader()), + instanceOf(expectedEventClass)); + } + assertEquals("more available", expectedIsMoreAvailable, + bufferAndBacklog.isMoreAvailable()); + assertEquals("more available", expectedIsMoreAvailable, readView.isAvailable()); + assertEquals("backlog", expectedBuffersInBacklog, bufferAndBacklog.buffersInBacklog()); + assertEquals("next is event", expectedNextBufferIsEvent, + bufferAndBacklog.nextBufferIsEvent()); + + assertFalse("not recycled", bufferAndBacklog.buffer().isRecycled()); + } finally { + bufferAndBacklog.buffer().recycleBuffer(); + } + assertEquals("recycled", expectedRecycledAfterRecycle, bufferAndBacklog.buffer().isRecycled()); } protected void assertNoNextBuffer(ResultSubpartitionView readView) throws IOException, InterruptedException { From d44e13dea76e4ff0f12f10f18916d27662723294 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 28 Feb 2018 09:42:38 +0100 Subject: [PATCH 106/268] [FLINK-8800][REST] Reduce logging of all requests to TRACE This closes #5594. --- .../java/org/apache/flink/runtime/rest/AbstractHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java index a259801d7a8fb..cb50a4f0d2ede 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/AbstractHandler.java @@ -84,8 +84,8 @@ protected AbstractHandler( @Override protected void respondAsLeader(ChannelHandlerContext ctx, Routed routed, T gateway) throws Exception { - if (log.isDebugEnabled()) { - log.debug("Received request " + routed.request().getUri() + '.'); + if (log.isTraceEnabled()) { + log.trace("Received request " + routed.request().getUri() + '.'); } final HttpRequest httpRequest = routed.request(); From 4d0b4db975577ccca50185bd5a1654a7fce5849d Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 6 Mar 2018 10:45:25 +0100 Subject: [PATCH 107/268] [FLINK-8847][build] Always generate .class files for package-info.java This closes #5644. --- pom.xml | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index dd1fa3729aa6f..34ac6c75fa1c5 100644 --- a/pom.xml +++ b/pom.xml @@ -1094,9 +1094,13 @@ under the License. ${java.version} ${java.version} - - -Xlint:all + + + -Xlint:all + + -Xpkginfo:always + From f6bbdcc2d3cff87dc4a5cd8e9100ac4fbf0a929a Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 6 Mar 2018 11:04:42 +0100 Subject: [PATCH 108/268] [hotfix][build] Enable incremental compilation --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index 34ac6c75fa1c5..887fb96e95cec 100644 --- a/pom.xml +++ b/pom.xml @@ -1094,6 +1094,8 @@ under the License. ${java.version} ${java.version} + + false From dbb8d2fb4a7fc058c7e3284602aed95ac9fb1d13 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 26 Feb 2018 10:12:44 +0100 Subject: [PATCH 109/268] [FLINK-8700] Add getters to JobDetailsInfo --- .../runtime/rest/messages/RequestBody.java | 2 +- .../runtime/rest/messages/ResponseBody.java | 2 +- .../rest/messages/job/JobDetailsInfo.java | 70 +++++++++++++++++++ 3 files changed, 72 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RequestBody.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RequestBody.java index ca55b17532ba3..8098ccfc1503f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RequestBody.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/RequestBody.java @@ -26,7 +26,7 @@ * *

    All fields that should part of the JSON request must be accessible either by being public or having a getter. * - *

    When adding methods that are prefixed with {@code get} make sure to annotate them with {@code @JsonIgnore}. + *

    When adding methods that are prefixed with {@code get/is} make sure to annotate them with {@code @JsonIgnore}. */ public interface RequestBody { } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ResponseBody.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ResponseBody.java index d4e94d1d6abdc..ff77966f4dcbe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ResponseBody.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ResponseBody.java @@ -26,7 +26,7 @@ * *

    All fields that should part of the JSON response must be accessible either by being public or having a getter. * - *

    When adding methods that are prefixed with {@code get} make sure to annotate them with {@code @JsonIgnore}. + *

    When adding methods that are prefixed with {@code get/is} make sure to annotate them with {@code @JsonIgnore}. */ public interface ResponseBody { } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java index 2c74389d96e8d..f839c182a5d84 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java @@ -32,6 +32,7 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonRawValue; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; @@ -165,6 +166,66 @@ public int hashCode() { return Objects.hash(jobId, name, isStoppable, jobStatus, startTime, endTime, duration, now, timestamps, jobVertexInfos, jobVerticesPerState, jsonPlan); } + @JsonIgnore + public JobID getJobId() { + return jobId; + } + + @JsonIgnore + public String getName() { + return name; + } + + @JsonIgnore + public boolean isStoppable() { + return isStoppable; + } + + @JsonIgnore + public JobStatus getJobStatus() { + return jobStatus; + } + + @JsonIgnore + public long getStartTime() { + return startTime; + } + + @JsonIgnore + public long getEndTime() { + return endTime; + } + + @JsonIgnore + public long getDuration() { + return duration; + } + + @JsonIgnore + public long getNow() { + return now; + } + + @JsonIgnore + public Map getTimestamps() { + return timestamps; + } + + @JsonIgnore + public Collection getJobVertexInfos() { + return jobVertexInfos; + } + + @JsonIgnore + public Map getJobVerticesPerState() { + return jobVerticesPerState; + } + + @JsonIgnore + public String getJsonPlan() { + return jsonPlan; + } + // --------------------------------------------------- // Static inner classes // --------------------------------------------------- @@ -242,38 +303,47 @@ public JobVertexDetailsInfo( this.jobVertexMetrics = Preconditions.checkNotNull(jobVertexMetrics); } + @JsonIgnore public JobVertexID getJobVertexID() { return jobVertexID; } + @JsonIgnore public String getName() { return name; } + @JsonIgnore public int getParallelism() { return parallelism; } + @JsonIgnore public ExecutionState getExecutionState() { return executionState; } + @JsonIgnore public long getStartTime() { return startTime; } + @JsonIgnore public long getEndTime() { return endTime; } + @JsonIgnore public long getDuration() { return duration; } + @JsonIgnore public Map getTasksPerState() { return tasksPerState; } + @JsonIgnore public IOMetricsInfo getJobVertexMetrics() { return jobVertexMetrics; } From 9e253b5d1ecaf0066d7aeae2828eee4f4671b2e8 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 26 Feb 2018 11:53:47 +0100 Subject: [PATCH 110/268] [FLINK-8700] Add ClusterClient.getJobStatus() --- .../flink/client/program/ClusterClient.java | 29 +++++++++++++++++++ .../client/program/MiniClusterClient.java | 1 + .../program/rest/RestClusterClient.java | 14 +++++++++ 3 files changed, 44 insertions(+) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index a4880db636930..1a783fc221384 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -49,6 +49,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -583,6 +584,34 @@ public JobListeningContext connectToJob(JobID jobID) throws JobExecutionExceptio printStatusDuringExecution); } + /** + * Requests the {@link JobStatus} of the job with the given {@link JobID}. + */ + public CompletableFuture getJobStatus(JobID jobId) { + final ActorGateway jobManager; + try { + jobManager = getJobManagerGateway(); + } catch (FlinkException e) { + throw new RuntimeException("Could not retrieve JobManage gateway.", e); + } + + Future response = jobManager.ask(JobManagerMessages.getRequestJobStatus(jobId), timeout); + + CompletableFuture javaFuture = FutureUtils.toJava(response); + + return javaFuture.thenApply((responseMessage) -> { + if (responseMessage instanceof JobManagerMessages.CurrentJobStatus) { + return ((JobManagerMessages.CurrentJobStatus) responseMessage).status(); + } else if (responseMessage instanceof JobManagerMessages.JobNotFound) { + throw new CompletionException( + new IllegalStateException("Could not find job with JobId " + jobId)); + } else { + throw new CompletionException( + new IllegalStateException("Unknown JobManager response of type " + responseMessage.getClass())); + } + }); + } + /** * Cancels a job identified by the job id. * @param jobId the job id diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index bbb5d49cbb8d3..7475071e23e6e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -137,6 +137,7 @@ public Map getAccumulators(JobID jobID, ClassLoader loader) thro throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); } + @Override public CompletableFuture getJobStatus(JobID jobId) { return guardWithSingleRetry(() -> miniCluster.getJobStatus(jobId), scheduledExecutor); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 18ff0992cb685..8cf0d2c7eb200 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -61,6 +62,8 @@ import org.apache.flink.runtime.rest.messages.ResponseBody; import org.apache.flink.runtime.rest.messages.TerminationModeQueryParameter; import org.apache.flink.runtime.rest.messages.TriggerId; +import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders; +import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo; import org.apache.flink.runtime.rest.messages.job.JobExecutionResultHeaders; import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders; import org.apache.flink.runtime.rest.messages.job.JobSubmitRequestBody; @@ -254,6 +257,17 @@ public JobSubmissionResult submitJob(JobGraph jobGraph, ClassLoader classLoader) } } + @Override + public CompletableFuture getJobStatus(JobID jobId) { + JobDetailsHeaders detailsHeaders = JobDetailsHeaders.getInstance(); + final JobMessageParameters params = new JobMessageParameters(); + params.jobPathParameter.resolve(jobId); + + CompletableFuture responseFuture = sendRequest(detailsHeaders, params); + + return responseFuture.thenApply(JobDetailsInfo::getJobStatus); + } + /** * Requests the {@link JobResult} for the given {@link JobID}. The method retries multiple * times to poll the {@link JobResult} before giving up. From c48612501b14174a04c7d1ad31c1e34de5e59854 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 28 Feb 2018 15:06:59 +0100 Subject: [PATCH 111/268] Add our own Deadline implementation --- .../flink/api/common/time/Deadline.java | 68 +++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/time/Deadline.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/time/Deadline.java b/flink-core/src/main/java/org/apache/flink/api/common/time/Deadline.java new file mode 100644 index 0000000000000..2db6579b10c62 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/time/Deadline.java @@ -0,0 +1,68 @@ +/* + * 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.api.common.time; + +import org.apache.flink.annotation.Internal; + +import java.time.Duration; + +/** + * This class stores a deadline, as obtained via {@link #now()} or from {@link #plus(Duration)}. + */ +@Internal +public class Deadline { + private final long timeNanos; + + private Deadline(Duration time) { + this.timeNanos = time.toNanos(); + } + + public Deadline plus(Duration other) { + return new Deadline(Duration.ofNanos(timeNanos).plus(other)); + } + + /** + * Returns the time left between the deadline and now. The result is negative if the deadline + * has passed. + */ + public Duration timeLeft() { + return Duration.ofNanos(timeNanos).minus(Duration.ofNanos(System.nanoTime())); + } + + /** + * Returns whether there is any time left between the deadline and now. + */ + public boolean hasTimeLeft() { + return !isOverdue(); + } + + /** + * Determines whether the deadline is in the past, i.e. whether the time left is negative. + */ + public boolean isOverdue() { + return timeNanos - System.nanoTime() < 0; + } + + /** + * Constructs a {@link Deadline} that has now as the deadline. Use this and then extend via + * {@link #plus(Duration)} to specify a deadline in the future. + */ + public static Deadline now() { + return new Deadline(Duration.ofNanos(System.nanoTime())); + } +} From 5d2788e378aedc6ee5133f080e70878d06e1743d Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 27 Feb 2018 13:40:51 +0100 Subject: [PATCH 112/268] [FLINK-8758] Add FutureUtils.retrySuccessfulWithDelay() This retries getting a result until it matches a given predicate or until we run out of retries. --- .../flink/runtime/concurrent/FutureUtils.java | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index da77bdc8e047f..a2d0710e87977 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.concurrent; +import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.util.ExceptionUtils; @@ -223,6 +224,81 @@ private static void retryOperationWithDelay( } } + /** + * Retry the given operation with the given delay in between successful completions where the + * result does not match a given predicate. + * + * @param operation to retry + * @param retryDelay delay between retries + * @param deadline A deadline that specifies at what point we should stop retrying + * @param acceptancePredicate Predicate to test whether the result is acceptable + * @param scheduledExecutor executor to be used for the retry operation + * @param type of the result + * @return Future which retries the given operation a given amount of times and delays the retry + * in case the predicate isn't matched + */ + public static CompletableFuture retrySuccesfulWithDelay( + final Supplier> operation, + final Time retryDelay, + final Deadline deadline, + final Predicate acceptancePredicate, + final ScheduledExecutor scheduledExecutor) { + + final CompletableFuture resultFuture = new CompletableFuture<>(); + + retrySuccessfulOperationWithDelay( + resultFuture, + operation, + retryDelay, + deadline, + acceptancePredicate, + scheduledExecutor); + + return resultFuture; + } + + private static void retrySuccessfulOperationWithDelay( + final CompletableFuture resultFuture, + final Supplier> operation, + final Time retryDelay, + final Deadline deadline, + final Predicate acceptancePredicate, + final ScheduledExecutor scheduledExecutor) { + + if (!resultFuture.isDone()) { + final CompletableFuture operationResultFuture = operation.get(); + + operationResultFuture.whenComplete( + (t, throwable) -> { + if (throwable != null) { + if (throwable instanceof CancellationException) { + resultFuture.completeExceptionally(new RetryException("Operation future was cancelled.", throwable)); + } else { + resultFuture.completeExceptionally(throwable); + } + } else { + if (acceptancePredicate.test(t)) { + resultFuture.complete(t); + } else if (deadline.hasTimeLeft()) { + final ScheduledFuture scheduledFuture = scheduledExecutor.schedule( + () -> retrySuccessfulOperationWithDelay(resultFuture, operation, retryDelay, deadline, acceptancePredicate, scheduledExecutor), + retryDelay.toMilliseconds(), + TimeUnit.MILLISECONDS); + + resultFuture.whenComplete( + (innerT, innerThrowable) -> scheduledFuture.cancel(false)); + } else { + resultFuture.completeExceptionally( + new RetryException("Could not satisfy the predicate within the allowed time.")); + } + } + }); + + resultFuture.whenComplete( + (t, throwable) -> operationResultFuture.cancel(false)); + } + } + /** * Exception with which the returned future is completed if the {@link #retry(Supplier, int, Executor)} * operation fails. From 7ab8ac352f5cedff8f95357c66d2631eee2fba52 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 27 Feb 2018 13:42:09 +0100 Subject: [PATCH 113/268] [FLINK-8797] Port AbstractOperatorRestoreTestBase to MiniClusterResource --- .../AbstractOperatorRestoreTestBase.java | 248 ++++++------------ 1 file changed, 81 insertions(+), 167 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index 9710c2080c077..72f700a82bcc4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -19,55 +19,40 @@ package org.apache.flink.test.state.operator.restore; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializers; -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.instance.AkkaActorGateway; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.runtime.jobmanager.JobManager; -import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.metrics.NoOpMetricRegistry; +import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.memory.MemoryStateBackend; -import org.apache.flink.runtime.taskmanager.TaskManager; -import org.apache.flink.runtime.testingUtils.TestingJobManager; -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; -import org.apache.flink.runtime.testingUtils.TestingMemoryArchivist; -import org.apache.flink.runtime.testingUtils.TestingTaskManager; -import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; +import org.apache.flink.test.util.MiniClusterResource; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.actor.PoisonPill; -import org.junit.AfterClass; -import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import java.io.File; import java.net.URL; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import scala.Option; -import scala.Tuple2; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; /** * Abstract class to verify that it is possible to migrate a savepoint across upgraded Flink versions and that the @@ -79,16 +64,21 @@ */ public abstract class AbstractOperatorRestoreTestBase extends TestLogger { + private static final int NUM_TMS = 1; + private static final int NUM_SLOTS_PER_TM = 4; + private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10000L); + @Rule public final TemporaryFolder tmpFolder = new TemporaryFolder(); - private static ActorSystem actorSystem = null; - private static HighAvailabilityServices highAvailabilityServices = null; - private static ActorGateway jobManager = null; - private static ActorGateway archiver = null; - private static ActorGateway taskManager = null; + @ClassRule + public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + new Configuration(), + NUM_TMS, + NUM_SLOTS_PER_TM), + true); - private static final FiniteDuration timeout = new FiniteDuration(30L, TimeUnit.SECONDS); private final boolean allowNonRestoredState; protected AbstractOperatorRestoreTestBase() { @@ -104,91 +94,21 @@ public static void beforeClass() { SavepointSerializers.setFailWhenLegacyStateDetected(false); } - @BeforeClass - public static void setupCluster() throws Exception { - final Configuration configuration = new Configuration(); - - FiniteDuration timeout = new FiniteDuration(30L, TimeUnit.SECONDS); - - actorSystem = AkkaUtils.createLocalActorSystem(new Configuration()); - - highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices( - configuration, - TestingUtils.defaultExecutor()); - - Tuple2 master = JobManager.startJobManagerActors( - configuration, - actorSystem, - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), - highAvailabilityServices, - NoOpMetricRegistry.INSTANCE, - Option.empty(), - Option.apply("jm"), - Option.apply("arch"), - TestingJobManager.class, - TestingMemoryArchivist.class); - - jobManager = LeaderRetrievalUtils.retrieveLeaderGateway( - highAvailabilityServices.getJobManagerLeaderRetriever(HighAvailabilityServices.DEFAULT_JOB_ID), - actorSystem, - timeout); - - archiver = new AkkaActorGateway(master._2(), jobManager.leaderSessionID()); - - Configuration tmConfig = new Configuration(); - tmConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4); - - ActorRef taskManagerRef = TaskManager.startTaskManagerComponentsAndActor( - tmConfig, - ResourceID.generate(), - actorSystem, - highAvailabilityServices, - NoOpMetricRegistry.INSTANCE, - "localhost", - Option.apply("tm"), - true, - TestingTaskManager.class); - - taskManager = new AkkaActorGateway(taskManagerRef, jobManager.leaderSessionID()); - - // Wait until connected - Object msg = new TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager(jobManager.actor()); - Await.ready(taskManager.ask(msg, timeout), timeout); - } - - @AfterClass - public static void tearDownCluster() throws Exception { - if (highAvailabilityServices != null) { - highAvailabilityServices.closeAndCleanupAllData(); - } - - if (actorSystem != null) { - actorSystem.shutdown(); - } - - if (archiver != null) { - archiver.actor().tell(PoisonPill.getInstance(), ActorRef.noSender()); - } - - if (jobManager != null) { - jobManager.actor().tell(PoisonPill.getInstance(), ActorRef.noSender()); - } - - if (taskManager != null) { - taskManager.actor().tell(PoisonPill.getInstance(), ActorRef.noSender()); - } - } - @Test public void testMigrationAndRestore() throws Throwable { + ClassLoader classLoader = this.getClass().getClassLoader(); + ClusterClient clusterClient = MINI_CLUSTER_RESOURCE.getClusterClient(); + clusterClient.setDetached(true); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); + // submit job with old version savepoint and create a migrated savepoint in the new version - String savepointPath = migrateJob(); + String savepointPath = migrateJob(classLoader, clusterClient, deadline); // restore from migrated new version savepoint - restoreJob(savepointPath); + restoreJob(classLoader, clusterClient, deadline, savepointPath); } - private String migrateJob() throws Throwable { + private String migrateJob(ClassLoader classLoader, ClusterClient clusterClient, Deadline deadline) throws Throwable { + URL savepointResource = AbstractOperatorRestoreTestBase.class.getClassLoader().getResource("operatorstate/" + getMigrationSavepointName()); if (savepointResource == null) { throw new IllegalArgumentException("Savepoint file does not exist."); @@ -196,86 +116,80 @@ private String migrateJob() throws Throwable { JobGraph jobToMigrate = createJobGraph(ExecutionMode.MIGRATE); jobToMigrate.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointResource.getFile())); - Object msg; - Object result; - - // Submit job graph - msg = new JobManagerMessages.SubmitJob(jobToMigrate, ListeningBehaviour.DETACHED); - result = Await.result(jobManager.ask(msg, timeout), timeout); + assertNotNull(jobToMigrate.getJobID()); - if (result instanceof JobManagerMessages.JobResultFailure) { - JobManagerMessages.JobResultFailure failure = (JobManagerMessages.JobResultFailure) result; - throw new Exception(failure.cause()); - } - Assert.assertSame(JobManagerMessages.JobSubmitSuccess.class, result.getClass()); + clusterClient.submitJob(jobToMigrate, classLoader); - // Wait for all tasks to be running - msg = new TestingJobManagerMessages.WaitForAllVerticesToBeRunning(jobToMigrate.getJobID()); - Await.result(jobManager.ask(msg, timeout), timeout); + CompletableFuture jobRunningFuture = FutureUtils.retrySuccesfulWithDelay( + () -> clusterClient.getJobStatus(jobToMigrate.getJobID()), + Time.milliseconds(50), + deadline, + (jobStatus) -> jobStatus == JobStatus.RUNNING, + TestingUtils.defaultScheduledExecutor()); + assertEquals( + JobStatus.RUNNING, + jobRunningFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); // Trigger savepoint File targetDirectory = tmpFolder.newFolder(); - msg = new JobManagerMessages.CancelJobWithSavepoint(jobToMigrate.getJobID(), targetDirectory.getAbsolutePath()); + String savepointPath = null; // FLINK-6918: Retry cancel with savepoint message in case that StreamTasks were not running // TODO: The retry logic should be removed once the StreamTask lifecycle has been fixed (see FLINK-4714) - boolean retry = true; - for (int i = 0; retry && i < 10; i++) { - Future future = jobManager.ask(msg, timeout); - result = Await.result(future, timeout); - - if (result instanceof JobManagerMessages.CancellationFailure) { - Thread.sleep(50L); - } else { - retry = false; + while (deadline.hasTimeLeft() && savepointPath == null) { + try { + savepointPath = clusterClient.cancelWithSavepoint( + jobToMigrate.getJobID(), + targetDirectory.getAbsolutePath()); + } catch (Exception e) { + String exceptionString = ExceptionUtils.stringifyException(e); + if (!(exceptionString.matches("(.*\n)*.*savepoint for the job .* failed(.*\n)*") // legacy + || exceptionString.matches("(.*\n)*.*Not all required tasks are currently running(.*\n)*") // flip6 + || exceptionString.matches("(.*\n)*.*Checkpoint was declined \\(tasks not ready\\)(.*\n)*"))) { // flip6 + throw e; + } } } - if (result instanceof JobManagerMessages.CancellationFailure) { - JobManagerMessages.CancellationFailure failure = (JobManagerMessages.CancellationFailure) result; - throw new Exception(failure.cause()); - } - - String savepointPath = ((JobManagerMessages.CancellationSuccess) result).savepointPath(); + assertNotNull("Could not take savepoint.", savepointPath); - // Wait until canceled - msg = new TestingJobManagerMessages.NotifyWhenJobStatus(jobToMigrate.getJobID(), JobStatus.CANCELED); - Await.ready(jobManager.ask(msg, timeout), timeout); + CompletableFuture jobCanceledFuture = FutureUtils.retrySuccesfulWithDelay( + () -> clusterClient.getJobStatus(jobToMigrate.getJobID()), + Time.milliseconds(50), + deadline, + (jobStatus) -> jobStatus == JobStatus.CANCELED, + TestingUtils.defaultScheduledExecutor()); + assertEquals( + JobStatus.CANCELED, + jobCanceledFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); return savepointPath; } - private void restoreJob(String savepointPath) throws Exception { + private void restoreJob(ClassLoader classLoader, ClusterClient clusterClient, Deadline deadline, String savepointPath) throws Exception { JobGraph jobToRestore = createJobGraph(ExecutionMode.RESTORE); jobToRestore.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath, allowNonRestoredState)); - Object msg; - Object result; - - // Submit job graph - msg = new JobManagerMessages.SubmitJob(jobToRestore, ListeningBehaviour.DETACHED); - result = Await.result(jobManager.ask(msg, timeout), timeout); + assertNotNull("Job doesn't have a JobID.", jobToRestore.getJobID()); - if (result instanceof JobManagerMessages.JobResultFailure) { - JobManagerMessages.JobResultFailure failure = (JobManagerMessages.JobResultFailure) result; - throw new Exception(failure.cause()); - } - Assert.assertSame(JobManagerMessages.JobSubmitSuccess.class, result.getClass()); - - msg = new JobManagerMessages.RequestJobStatus(jobToRestore.getJobID()); - JobStatus status = ((JobManagerMessages.CurrentJobStatus) Await.result(jobManager.ask(msg, timeout), timeout)).status(); - while (!status.isTerminalState()) { - status = ((JobManagerMessages.CurrentJobStatus) Await.result(jobManager.ask(msg, timeout), timeout)).status(); - } + clusterClient.submitJob(jobToRestore, classLoader); - Assert.assertEquals(JobStatus.FINISHED, status); + CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> clusterClient.getJobStatus(jobToRestore.getJobID()), + Time.milliseconds(50), + deadline, + (jobStatus) -> jobStatus == JobStatus.FINISHED, + TestingUtils.defaultScheduledExecutor()); + assertEquals( + JobStatus.FINISHED, + jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); } private JobGraph createJobGraph(ExecutionMode mode) { - StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(500, CheckpointingMode.EXACTLY_ONCE); env.setRestartStrategy(RestartStrategies.noRestart()); - env.setStateBackend(new MemoryStateBackend()); + env.setStateBackend((StateBackend) new MemoryStateBackend()); switch (mode) { case MIGRATE: From 735c95fbb93316475dc6dd6daa2aa91a7158c843 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 26 Feb 2018 11:55:14 +0100 Subject: [PATCH 114/268] [FLINK-8778] Port queryable state ITCases to use MiniClusterResource --- .../AbstractQueryableStateTestBase.java | 240 ++++++++---------- .../HAAbstractQueryableStateTestBase.java | 93 ------- .../HAQueryableStateFsBackendITCase.java | 90 ++++++- .../HAQueryableStateRocksDBBackendITCase.java | 91 ++++++- .../NonHAAbstractQueryableStateTestBase.java | 75 ------ .../NonHAQueryableStateFsBackendITCase.java | 60 ++++- ...nHAQueryableStateRocksDBBackendITCase.java | 61 ++++- 7 files changed, 375 insertions(+), 335 deletions(-) delete mode 100644 flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java delete mode 100644 flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index 623e42b43e010..e99a28b36a212 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -37,12 +37,15 @@ import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.configuration.Configuration; import org.apache.flink.queryablestate.client.QueryableStateClient; import org.apache.flink.queryablestate.client.VoidNamespace; @@ -53,12 +56,9 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.messages.JobManagerMessages.CancellationSuccess; -import org.apache.flink.runtime.minicluster.FlinkMiniCluster; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.CheckpointListener; -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; +import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.QueryableStateStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -68,6 +68,7 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.Collector; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -76,6 +77,7 @@ import org.junit.Ignore; import org.junit.Test; +import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -93,11 +95,9 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLongArray; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; -import scala.reflect.ClassTag$; - +import static org.hamcrest.CoreMatchers.containsString; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -106,7 +106,7 @@ */ public abstract class AbstractQueryableStateTestBase extends TestLogger { - private static final FiniteDuration TEST_TIMEOUT = new FiniteDuration(10000L, TimeUnit.SECONDS); + private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10000L); public static final long RETRY_TIMEOUT = 50L; private final ScheduledExecutorService executorService = Executors.newScheduledThreadPool(4); @@ -117,28 +117,23 @@ public abstract class AbstractQueryableStateTestBase extends TestLogger { */ protected AbstractStateBackend stateBackend; - /** - * Shared between all the test. Make sure to have at least NUM_SLOTS - * available after your test finishes, e.g. cancel the job you submitted. - */ - protected static FlinkMiniCluster cluster; - /** * Client shared between all the test. */ protected static QueryableStateClient client; + protected static ClusterClient clusterClient; + protected static int maxParallelism; @Before public void setUp() throws Exception { - // NOTE: do not use a shared instance for all tests as the tests may brake + // NOTE: do not use a shared instance for all tests as the tests may break this.stateBackend = createStateBackend(); - Assert.assertNotNull(cluster); + Assert.assertNotNull(clusterClient); - maxParallelism = cluster.configuration().getInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1) * - cluster.configuration().getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + maxParallelism = 4; } /** @@ -160,8 +155,7 @@ public void setUp() throws Exception { @Test @SuppressWarnings("unchecked") public void testQueryableState() throws Exception { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final int numKeys = 256; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -188,12 +182,13 @@ public Integer getKey(Tuple2 value) { } }).asQueryableState(queryName, reducingState); - try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(deadline, clusterClient, env)) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); final AtomicLongArray counts = new AtomicLongArray(numKeys); @@ -257,9 +252,8 @@ public Integer getKey(Tuple2 value) { /** * Tests that duplicate query registrations fail the job at the JobManager. */ - @Test + @Test(timeout = 60_000) public void testDuplicateRegistrationFailsJob() throws Exception { - final Deadline deadline = TEST_TIMEOUT.fromNow(); final int numKeys = 256; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -302,54 +296,19 @@ public Integer getKey(Tuple2 value) { // Submit the job graph final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - final JobID jobId = jobGraph.getJobID(); - - final CompletableFuture failedFuture = - notifyWhenJobStatusIs(jobId, JobStatus.FAILED, deadline); - final CompletableFuture cancellationFuture = - notifyWhenJobStatusIs(jobId, JobStatus.CANCELED, deadline); - - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(false); + boolean caughtException = false; try { - final TestingJobManagerMessages.JobStatusIs jobStatus = - failedFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - - assertEquals(JobStatus.FAILED, jobStatus.state()); - } catch (Exception e) { - - // if the assertion fails, it means that the job was (falsely) not cancelled. - // in this case, and given that the mini-cluster is shared with other tests, - // we cancel the job and wait for the cancellation so that the resources are freed. - - if (jobId != null) { - cluster.getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class)); - - cancellationFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } - - // and we re-throw the exception. - throw e; + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); + } catch (ProgramInvocationException e) { + String failureCause = ExceptionUtils.stringifyException(e); + assertThat(failureCause, containsString("KvState with name '" + queryName + "' has already been registered by another operator")); + caughtException = true; } - // Get the job and check the cause - JobManagerMessages.JobFound jobFound = FutureUtils.toJava( - cluster.getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.RequestJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(JobManagerMessages.JobFound.class))) - .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - - String failureCause = jobFound.executionGraph().getFailureInfo().getExceptionAsString(); - - assertEquals(JobStatus.FAILED, jobFound.executionGraph().getState()); - assertTrue("Not instance of SuppressRestartsException", failureCause.startsWith("org.apache.flink.runtime.execution.SuppressRestartsException")); - int causedByIndex = failureCause.indexOf("Caused by: "); - String subFailureCause = failureCause.substring(causedByIndex + "Caused by: ".length()); - assertTrue("Not caused by IllegalStateException", subFailureCause.startsWith("java.lang.IllegalStateException")); - assertTrue("Exception does not contain registration name", subFailureCause.contains(queryName)); + assertTrue(caughtException); } /** @@ -360,8 +319,7 @@ public Integer getKey(Tuple2 value) { */ @Test public void testValueState() throws Exception { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final long numElements = 1024L; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -386,12 +344,13 @@ public Integer getKey(Tuple2 value) { } }).asQueryableState("hakuna", valueState); - try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(deadline, clusterClient, env)) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); executeValueQuery(deadline, client, jobId, "hakuna", valueState, numElements); } @@ -404,8 +363,7 @@ public Integer getKey(Tuple2 value) { @Test @Ignore public void testWrongJobIdAndWrongQueryableStateName() throws Exception { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final long numElements = 1024L; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -425,18 +383,22 @@ public Integer getKey(Tuple2 value) { } }).asQueryableState("hakuna", valueState); - try (AutoCancellableJob closableJobGraph = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob closableJobGraph = new AutoCancellableJob(deadline, clusterClient, env)) { - // register to be notified when the job is running. - CompletableFuture runningFuture = - notifyWhenJobStatusIs(closableJobGraph.getJobId(), JobStatus.RUNNING, deadline); + clusterClient.setDetached(true); + clusterClient.submitJob( + closableJobGraph.getJobGraph(), AbstractQueryableStateTestBase.class.getClassLoader()); - cluster.submitJobDetached(closableJobGraph.getJobGraph()); + CompletableFuture jobStatusFuture = + clusterClient.getJobStatus(closableJobGraph.getJobId()); + + while (deadline.hasTimeLeft() && !jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS).equals(JobStatus.RUNNING)) { + Thread.sleep(50); + jobStatusFuture = + clusterClient.getJobStatus(closableJobGraph.getJobId()); + } - // expect for the job to be running - TestingJobManagerMessages.JobStatusIs jobStatus = - runningFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - assertEquals(JobStatus.RUNNING, jobStatus.state()); + assertEquals(JobStatus.RUNNING, jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); final JobID wrongJobId = new JobID(); @@ -484,14 +446,13 @@ public Integer getKey(Tuple2 value) { */ @Test public void testQueryNonStartedJobState() throws Exception { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final long numElements = 1024L; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStateBackend(stateBackend); env.setParallelism(maxParallelism); - // Very important, because cluster is shared between tests and we + // Very important, because clusterClient is shared between tests and we // don't explicitly check that all slots are available before // submitting. env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 1000L)); @@ -512,7 +473,7 @@ public Integer getKey(Tuple2 value) { } }).asQueryableState("hakuna", valueState); - try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(deadline, clusterClient, env)) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); @@ -527,7 +488,8 @@ public Integer getKey(Tuple2 value) { BasicTypeInfo.INT_TYPE_INFO, valueState); - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); executeValueQuery(deadline, client, jobId, "hakuna", valueState, expected); } @@ -543,8 +505,7 @@ public Integer getKey(Tuple2 value) { */ @Test(expected = UnknownKeyOrNamespaceException.class) public void testValueStateDefault() throws Throwable { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final long numElements = 1024L; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -571,12 +532,13 @@ public Integer getKey(Tuple2 value) { } }).asQueryableState("hakuna", valueState); - try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(deadline, clusterClient, env)) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); // Now query int key = 0; @@ -611,8 +573,7 @@ public Integer getKey(Tuple2 value) { */ @Test public void testValueStateShortcut() throws Exception { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final long numElements = 1024L; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -639,12 +600,14 @@ public Integer getKey(Tuple2 value) { final ValueStateDescriptor> stateDesc = (ValueStateDescriptor>) queryableState.getStateDescriptor(); - try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(deadline, clusterClient, env)) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); + executeValueQuery(deadline, client, jobId, "matata", stateDesc, numElements); } } @@ -658,8 +621,7 @@ public Integer getKey(Tuple2 value) { */ @Test public void testFoldingState() throws Exception { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final int numElements = 1024; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -684,12 +646,13 @@ public Integer getKey(Tuple2 value) { } }).asQueryableState("pumba", foldingState); - try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(deadline, clusterClient, env)) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); final String expected = Integer.toString(numElements * (numElements + 1) / 2); @@ -731,8 +694,7 @@ public Integer getKey(Tuple2 value) { */ @Test public void testReducingState() throws Exception { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final long numElements = 1024L; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -757,12 +719,13 @@ public Integer getKey(Tuple2 value) { } }).asQueryableState("jungle", reducingState); - try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(deadline, clusterClient, env)) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); final long expected = numElements * (numElements + 1L) / 2L; @@ -804,8 +767,7 @@ public Integer getKey(Tuple2 value) { */ @Test public void testMapState() throws Exception { - - final Deadline deadline = TEST_TIMEOUT.fromNow(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); final long numElements = 1024L; StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -850,12 +812,13 @@ public void processElement(Tuple2 value, Context ctx, Collector value, Context ctx, Collector value, Context ctx, Collector> results = new HashMap<>(); @@ -994,8 +957,7 @@ public void processElement(Tuple2 value, Context ctx, Collector value) { new AggregatingTestOperator(aggrStateDescriptor) ); - try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(cluster, env, deadline)) { + try (AutoCancellableJob autoCancellableJob = new AutoCancellableJob(deadline, clusterClient, env)) { final JobID jobId = autoCancellableJob.getJobId(); final JobGraph jobGraph = autoCancellableJob.getJobGraph(); - cluster.submitJobDetached(jobGraph); + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, AbstractQueryableStateTestBase.class.getClassLoader()); for (int key = 0; key < maxParallelism; key++) { boolean success = false; @@ -1277,22 +1240,22 @@ public Tuple2 reduce(Tuple2 value1, Tuple2 clusterClient; private final JobGraph jobGraph; private final JobID jobId; - private final CompletableFuture cancellationFuture; - AutoCancellableJob(final FlinkMiniCluster cluster, final StreamExecutionEnvironment env, final Deadline deadline) { + private final Deadline deadline; + + AutoCancellableJob(Deadline deadline, final ClusterClient clusterClient, final StreamExecutionEnvironment env) { Preconditions.checkNotNull(env); - this.cluster = Preconditions.checkNotNull(cluster); + this.clusterClient = Preconditions.checkNotNull(clusterClient); this.jobGraph = env.getStreamGraph().getJobGraph(); - this.deadline = Preconditions.checkNotNull(deadline); - this.jobId = jobGraph.getJobID(); - this.cancellationFuture = notifyWhenJobStatusIs(jobId, JobStatus.CANCELED, deadline); + this.jobId = Preconditions.checkNotNull(jobGraph.getJobID()); + + this.deadline = deadline; } JobGraph getJobGraph() { @@ -1306,25 +1269,20 @@ JobID getJobId() { @Override public void close() throws Exception { // Free cluster resources - if (jobId != null) { - cluster.getLeaderGateway(deadline.timeLeft()) - .ask(new JobManagerMessages.CancelJob(jobId), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(CancellationSuccess.class)); - - cancellationFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - } + clusterClient.cancel(jobId); + // cancel() is non-blocking so do this to make sure the job finished + CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> clusterClient.getJobStatus(jobId), + Time.milliseconds(50), + deadline, + (jobStatus) -> jobStatus.equals(JobStatus.CANCELED), + TestingUtils.defaultScheduledExecutor()); + assertEquals( + JobStatus.CANCELED, + jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); } } - private static CompletableFuture notifyWhenJobStatusIs( - final JobID jobId, final JobStatus status, final Deadline deadline) { - - return FutureUtils.toJava( - cluster.getLeaderGateway(deadline.timeLeft()) - .ask(new TestingJobManagerMessages.NotifyWhenJobStatus(jobId, status), deadline.timeLeft()) - .mapTo(ClassTag$.MODULE$.apply(TestingJobManagerMessages.JobStatusIs.class))); - } - private static CompletableFuture getKvState( final Deadline deadline, final QueryableStateClient client, diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java deleted file mode 100644 index 8767b5214e98d..0000000000000 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAAbstractQueryableStateTestBase.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * 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.queryablestate.itcases; - -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.configuration.QueryableStateOptions; -import org.apache.flink.queryablestate.client.QueryableStateClient; -import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.runtime.testingUtils.TestingCluster; - -import org.apache.curator.test.TestingServer; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; - -import static org.junit.Assert.fail; - -/** - * Base class with the cluster configuration for the tests on the NON-HA mode. - */ -public abstract class HAAbstractQueryableStateTestBase extends AbstractQueryableStateTestBase { - - private static final int NUM_JMS = 2; - private static final int NUM_TMS = 2; - private static final int NUM_SLOTS_PER_TM = 4; - - private static TestingServer zkServer; - private static TemporaryFolder temporaryFolder; - - public static void setup(int proxyPortRangeStart, int serverPortRangeStart) { - try { - zkServer = new TestingServer(); - temporaryFolder = new TemporaryFolder(); - temporaryFolder.create(); - - Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); - config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 2); - config.setInteger(QueryableStateOptions.PROXY_NETWORK_THREADS, 2); - config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 2); - config.setString(QueryableStateOptions.PROXY_PORT_RANGE, proxyPortRangeStart + "-" + (proxyPortRangeStart + NUM_TMS)); - config.setString(QueryableStateOptions.SERVER_PORT_RANGE, serverPortRangeStart + "-" + (serverPortRangeStart + NUM_TMS)); - config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().toString()); - config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); - config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); - - cluster = new TestingCluster(config, false); - cluster.start(true); - - client = new QueryableStateClient("localhost", proxyPortRangeStart); - - // verify that we are in HA mode - Assert.assertTrue(cluster.haMode() == HighAvailabilityMode.ZOOKEEPER); - - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @AfterClass - public static void tearDown() throws IOException { - client.shutdownAndWait(); - - cluster.stop(); - cluster.awaitTermination(); - - zkServer.stop(); - zkServer.close(); - } -} diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java index 6f31e76b5aa1f..a47045f35a8c5 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateFsBackendITCase.java @@ -18,28 +18,102 @@ package org.apache.flink.queryablestate.itcases; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.QueryableStateOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.WebOptions; +import org.apache.flink.queryablestate.client.QueryableStateClient; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.test.util.MiniClusterResource; +import org.apache.curator.test.TestingServer; +import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; /** * Several integration tests for queryable state using the {@link FsStateBackend}. */ -public class HAQueryableStateFsBackendITCase extends HAAbstractQueryableStateTestBase { +public class HAQueryableStateFsBackendITCase extends AbstractQueryableStateTestBase { - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static final int NUM_JMS = 2; + // NUM_TMS * NUM_SLOTS_PER_TM must match the parallelism of the pipelines so that + // we always use all TaskManagers so that the JM oracle is always properly re-registered + private static final int NUM_TMS = 2; + private static final int NUM_SLOTS_PER_TM = 2; - @BeforeClass - public static void setup() { - setup(9064, 9069); - } + private static final int QS_PROXY_PORT_RANGE_START = 9064; + private static final int QS_SERVER_PORT_RANGE_START = 9069; + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static TestingServer zkServer; + + private static MiniClusterResource miniClusterResource; @Override protected AbstractStateBackend createStateBackend() throws Exception { return new FsStateBackend(temporaryFolder.newFolder().toURI().toString()); } + + @BeforeClass + public static void setup() throws Exception { + zkServer = new TestingServer(); + + // we have to manage this manually because we have to create the ZooKeeper server + // ahead of this + miniClusterResource = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfig(), + NUM_TMS, + NUM_SLOTS_PER_TM), + true); + + miniClusterResource.before(); + + client = new QueryableStateClient("localhost", QS_PROXY_PORT_RANGE_START); + + clusterClient = miniClusterResource.getClusterClient(); + } + + @AfterClass + public static void tearDown() throws Exception { + miniClusterResource.after(); + + client.shutdownAndWait(); + + zkServer.stop(); + zkServer.close(); + } + + private static Configuration getConfig() throws Exception { + + Configuration config = new Configuration(); + config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L); + config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); + config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 2); + config.setInteger(QueryableStateOptions.PROXY_NETWORK_THREADS, 2); + config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 2); + config.setString( + QueryableStateOptions.PROXY_PORT_RANGE, + QS_PROXY_PORT_RANGE_START + "-" + (QS_PROXY_PORT_RANGE_START + NUM_TMS)); + config.setString( + QueryableStateOptions.SERVER_PORT_RANGE, + QS_SERVER_PORT_RANGE_START + "-" + (QS_SERVER_PORT_RANGE_START + NUM_TMS)); + config.setBoolean(WebOptions.SUBMIT_ENABLE, false); + + config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().toString()); + + config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); + config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); + + return config; + } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java index cae02e2ba69c8..b1092c1416702 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/HAQueryableStateRocksDBBackendITCase.java @@ -18,28 +18,103 @@ package org.apache.flink.queryablestate.itcases; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.QueryableStateOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.WebOptions; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; +import org.apache.flink.queryablestate.client.QueryableStateClient; import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.test.util.MiniClusterResource; +import org.apache.curator.test.TestingServer; +import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; /** * Several integration tests for queryable state using the {@link RocksDBStateBackend}. */ -public class HAQueryableStateRocksDBBackendITCase extends HAAbstractQueryableStateTestBase { +public class HAQueryableStateRocksDBBackendITCase extends AbstractQueryableStateTestBase { - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); + private static final int NUM_JMS = 2; + // NUM_TMS * NUM_SLOTS_PER_TM must match the parallelism of the pipelines so that + // we always use all TaskManagers so that the JM oracle is always properly re-registered + private static final int NUM_TMS = 2; + private static final int NUM_SLOTS_PER_TM = 2; - @BeforeClass - public static void setup() { - setup(9074, 9079); - } + private static final int QS_PROXY_PORT_RANGE_START = 9074; + private static final int QS_SERVER_PORT_RANGE_START = 9079; + + @ClassRule + public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private static TestingServer zkServer; + + private static MiniClusterResource miniClusterResource; @Override protected AbstractStateBackend createStateBackend() throws Exception { return new RocksDBStateBackend(temporaryFolder.newFolder().toURI().toString()); } + + @BeforeClass + public static void setup() throws Exception { + zkServer = new TestingServer(); + + // we have to manage this manually because we have to create the ZooKeeper server + // ahead of this + miniClusterResource = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfig(), + NUM_TMS, + NUM_SLOTS_PER_TM), + true); + + miniClusterResource.before(); + + client = new QueryableStateClient("localhost", QS_PROXY_PORT_RANGE_START); + + clusterClient = miniClusterResource.getClusterClient(); + } + + @AfterClass + public static void tearDown() throws Exception { + miniClusterResource.after(); + + client.shutdownAndWait(); + + zkServer.stop(); + zkServer.close(); + } + + private static Configuration getConfig() throws Exception { + + Configuration config = new Configuration(); + config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L); + config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); + config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 2); + config.setInteger(QueryableStateOptions.PROXY_NETWORK_THREADS, 2); + config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 2); + config.setString( + QueryableStateOptions.PROXY_PORT_RANGE, + QS_PROXY_PORT_RANGE_START + "-" + (QS_PROXY_PORT_RANGE_START + NUM_TMS)); + config.setString( + QueryableStateOptions.SERVER_PORT_RANGE, + QS_SERVER_PORT_RANGE_START + "-" + (QS_SERVER_PORT_RANGE_START + NUM_TMS)); + config.setBoolean(WebOptions.SUBMIT_ENABLE, false); + + config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().toString()); + + config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); + config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); + + return config; + } + } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java deleted file mode 100644 index 2686a2981f319..0000000000000 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAAbstractQueryableStateTestBase.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.queryablestate.itcases; - -import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.QueryableStateOptions; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.queryablestate.client.QueryableStateClient; -import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.runtime.testingUtils.TestingCluster; - -import org.junit.AfterClass; -import org.junit.Assert; - -import static org.junit.Assert.fail; - -/** - * Base class with the cluster configuration for the tests on the HA mode. - */ -public abstract class NonHAAbstractQueryableStateTestBase extends AbstractQueryableStateTestBase { - - private static final int NUM_TMS = 2; - private static final int NUM_SLOTS_PER_TM = 4; - - public static void setup(int proxyPortRangeStart, int serverPortRangeStart) { - try { - Configuration config = new Configuration(); - config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); - config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1); - config.setInteger(QueryableStateOptions.PROXY_NETWORK_THREADS, 1); - config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 1); - config.setString(QueryableStateOptions.PROXY_PORT_RANGE, proxyPortRangeStart + "-" + (proxyPortRangeStart + NUM_TMS)); - config.setString(QueryableStateOptions.SERVER_PORT_RANGE, serverPortRangeStart + "-" + (serverPortRangeStart + NUM_TMS)); - - cluster = new TestingCluster(config, false); - cluster.start(true); - - client = new QueryableStateClient("localhost", proxyPortRangeStart); - - // verify that we are not in HA mode - Assert.assertTrue(cluster.haMode() == HighAvailabilityMode.NONE); - - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } - - @AfterClass - public static void tearDown() { - client.shutdownAndWait(); - - cluster.stop(); - cluster.awaitTermination(); - } -} diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java index 9457e0f047127..eb300c12e4cab 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateFsBackendITCase.java @@ -18,28 +18,78 @@ package org.apache.flink.queryablestate.itcases; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.QueryableStateOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.WebOptions; +import org.apache.flink.queryablestate.client.QueryableStateClient; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.test.util.MiniClusterResource; +import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.rules.TemporaryFolder; /** * Several integration tests for queryable state using the {@link FsStateBackend}. */ -public class NonHAQueryableStateFsBackendITCase extends NonHAAbstractQueryableStateTestBase { +public class NonHAQueryableStateFsBackendITCase extends AbstractQueryableStateTestBase { + + // NUM_TMS * NUM_SLOTS_PER_TM must match the parallelism of the pipelines so that + // we always use all TaskManagers so that the JM oracle is always properly re-registered + private static final int NUM_TMS = 2; + private static final int NUM_SLOTS_PER_TM = 2; + + private static final int QS_PROXY_PORT_RANGE_START = 9084; + private static final int QS_SERVER_PORT_RANGE_START = 9089; @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - @BeforeClass - public static void setup() { - setup(9084, 9089); - } + @ClassRule + public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfig(), + NUM_TMS, + NUM_SLOTS_PER_TM), + true); @Override protected AbstractStateBackend createStateBackend() throws Exception { return new FsStateBackend(temporaryFolder.newFolder().toURI().toString()); } + + @BeforeClass + public static void setup() throws Exception { + client = new QueryableStateClient("localhost", QS_PROXY_PORT_RANGE_START); + + clusterClient = MINI_CLUSTER_RESOURCE.getClusterClient(); + } + + @AfterClass + public static void tearDown() { + client.shutdownAndWait(); + } + + private static Configuration getConfig() { + Configuration config = new Configuration(); + config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); + config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1); + config.setInteger(QueryableStateOptions.PROXY_NETWORK_THREADS, 1); + config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 1); + config.setString( + QueryableStateOptions.PROXY_PORT_RANGE, + QS_PROXY_PORT_RANGE_START + "-" + (QS_PROXY_PORT_RANGE_START + NUM_TMS)); + config.setString( + QueryableStateOptions.SERVER_PORT_RANGE, + QS_SERVER_PORT_RANGE_START + "-" + (QS_SERVER_PORT_RANGE_START + NUM_TMS)); + config.setBoolean(WebOptions.SUBMIT_ENABLE, false); + return config; + } } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java index 7778a9446bd9d..3d6a3e3fcfdc0 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/NonHAQueryableStateRocksDBBackendITCase.java @@ -18,28 +18,79 @@ package org.apache.flink.queryablestate.itcases; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.QueryableStateOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.configuration.WebOptions; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; +import org.apache.flink.queryablestate.client.QueryableStateClient; import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.test.util.MiniClusterResource; +import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.rules.TemporaryFolder; /** * Several integration tests for queryable state using the {@link RocksDBStateBackend}. */ -public class NonHAQueryableStateRocksDBBackendITCase extends NonHAAbstractQueryableStateTestBase { +public class NonHAQueryableStateRocksDBBackendITCase extends AbstractQueryableStateTestBase { + + // NUM_TMS * NUM_SLOTS_PER_TM must match the parallelism of the pipelines so that + // we always use all TaskManagers so that the JM oracle is always properly re-registered + private static final int NUM_TMS = 2; + private static final int NUM_SLOTS_PER_TM = 2; + + private static final int QS_PROXY_PORT_RANGE_START = 9094; + private static final int QS_SERVER_PORT_RANGE_START = 9099; @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - @BeforeClass - public static void setup() { - setup(9094, 9099); - } + @ClassRule + public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfig(), + NUM_TMS, + NUM_SLOTS_PER_TM), + true); @Override protected AbstractStateBackend createStateBackend() throws Exception { return new RocksDBStateBackend(temporaryFolder.newFolder().toURI().toString()); } + + @BeforeClass + public static void setup() throws Exception { + client = new QueryableStateClient("localhost", QS_PROXY_PORT_RANGE_START); + + clusterClient = MINI_CLUSTER_RESOURCE.getClusterClient(); + } + + @AfterClass + public static void tearDown() { + client.shutdownAndWait(); + } + + private static Configuration getConfig() { + Configuration config = new Configuration(); + config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); + config.setInteger(QueryableStateOptions.CLIENT_NETWORK_THREADS, 1); + config.setInteger(QueryableStateOptions.PROXY_NETWORK_THREADS, 1); + config.setInteger(QueryableStateOptions.SERVER_NETWORK_THREADS, 1); + config.setString( + QueryableStateOptions.PROXY_PORT_RANGE, + QS_PROXY_PORT_RANGE_START + "-" + (QS_PROXY_PORT_RANGE_START + NUM_TMS)); + config.setString( + QueryableStateOptions.SERVER_PORT_RANGE, + QS_SERVER_PORT_RANGE_START + "-" + (QS_SERVER_PORT_RANGE_START + NUM_TMS)); + config.setBoolean(WebOptions.SUBMIT_ENABLE, false); + return config; + } + } From b4136d24831fe00d95e6d840748a6ee59b5336f8 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 9 Mar 2018 14:11:04 +0100 Subject: [PATCH 115/268] [FLINK-8911] Add separate script for nightly end-to-end tests --- flink-end-to-end-tests/README.md | 18 +++++++- flink-end-to-end-tests/run-nightly-tests.sh | 51 +++++++++++++++++++++ 2 files changed, 67 insertions(+), 2 deletions(-) create mode 100755 flink-end-to-end-tests/run-nightly-tests.sh diff --git a/flink-end-to-end-tests/README.md b/flink-end-to-end-tests/README.md index 1c8aadcc6177b..82d7e419ad8f9 100644 --- a/flink-end-to-end-tests/README.md +++ b/flink-end-to-end-tests/README.md @@ -1,14 +1,28 @@ # Flink End-to-End Tests -This module contains tests that verify end-to-end behaviour of Flink. +This module contains tests that verify end-to-end behaviour of Flink. We +categorize end-to-end tests as either pre-commit tests or nightly tests. The +former should be run on every commit, that is every Travis run, while the second +category should be run by a nightly job or when manually verifying a release or +making sure that the tests pass. + +Tests in the pre-commit category should be more lightweight while tests in the +nightly category can be quite heavyweight because we don't run them for every +commit. ## Running Tests -You can run all tests by executing +You can run all pre-commit tests by executing ``` $ FLINK_DIR= flink-end-to-end-tests/run-pre-commit-tests.sh ``` +and all nightly tests via + +``` +$ FLINK_DIR= flink-end-to-end-tests/run-nightly-tests.sh +``` + where is a Flink distribution directory. You can also run tests individually via diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh new file mode 100755 index 0000000000000..8ee526bc23ed0 --- /dev/null +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +END_TO_END_DIR="`dirname \"$0\"`" # relative +END_TO_END_DIR="`( cd \"$END_TO_END_DIR\" && pwd )`" # absolutized and normalized +if [ -z "$END_TO_END_DIR" ] ; then + # error; for some reason, the path is not accessible + # to the script (e.g. permissions re-evaled after suid) + exit 1 # fail +fi + +if [ -z "$FLINK_DIR" ] ; then + echo "You have to export the Flink distribution directory as FLINK_DIR" + exit 1 +fi + +FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd )`" # absolutized and normalized + +echo "flink-end-to-end-test directory: $END_TO_END_DIR" +echo "Flink distribution directory: $FLINK_DIR" + +EXIT_CODE=0 + +# Template for adding a test: + +# if [ $EXIT_CODE == 0 ]; then +# printf "\n==============================================================================\n" +# printf "Running my fancy nightly end-to-end test\n" +# printf "==============================================================================\n" +# $END_TO_END_DIR/test-scripts/test_something_very_fancy.sh +# EXIT_CODE=$? +# fi + +# Exit code for Travis build success/failure +exit $EXIT_CODE From b4f9e61b5d061f2fa9166bc5ea83ef6cd80eb0e6 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Sat, 3 Mar 2018 09:34:56 +0100 Subject: [PATCH 116/268] [FLINK-8487] Verify ZooKeeper checkpoint store behaviour with ITCase --- .../ZooKeeperCompletedCheckpointStore.java | 2 +- .../ZooKeeperHighAvailabilityITCase.java | 326 ++++++++++++++++++ 2 files changed, 327 insertions(+), 1 deletion(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java index 0cbd4fb6c9e9f..f22127041d31b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStore.java @@ -209,7 +209,7 @@ public void recover() throws Exception { if (completedCheckpoints.isEmpty() && numberOfInitialCheckpoints > 0) { throw new FlinkException( - "Could not read any of the " + numberOfInitialCheckpoints + " from storage."); + "Could not read any of the " + numberOfInitialCheckpoints + " checkpoints from storage."); } else if (completedCheckpoints.size() != numberOfInitialCheckpoints) { LOG.warn( "Could only fetch {} of {} checkpoints from storage.", diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java new file mode 100644 index 0000000000000..156d4486c627b --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ZooKeeperHighAvailabilityITCase.java @@ -0,0 +1,326 @@ +/* + * 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.test.checkpointing; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.runtime.state.StateBackend; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.test.util.MiniClusterResource; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLogger; + +import org.apache.curator.test.TestingServer; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.time.Duration; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +/** + * Integration tests for {@link org.apache.flink.runtime.checkpoint.ZooKeeperCompletedCheckpointStore}. + */ +public class ZooKeeperHighAvailabilityITCase extends TestLogger { + + private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10000L); + + private static final int NUM_JMS = 1; + private static final int NUM_TMS = 1; + private static final int NUM_SLOTS_PER_TM = 1; + + @ClassRule + public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static File haStorageDir; + + private static TestingServer zkServer; + + private static MiniClusterResource miniClusterResource; + + private static OneShotLatch waitForCheckpointLatch = new OneShotLatch(); + private static OneShotLatch failInCheckpointLatch = new OneShotLatch(); + + @BeforeClass + public static void setup() throws Exception { + zkServer = new TestingServer(); + + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, NUM_JMS); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); + + haStorageDir = TEMPORARY_FOLDER.newFolder(); + + config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haStorageDir.toString()); + config.setString(HighAvailabilityOptions.HA_CLUSTER_ID, UUID.randomUUID().toString()); + config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); + config.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); + + // we have to manage this manually because we have to create the ZooKeeper server + // ahead of this + miniClusterResource = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + config, + NUM_TMS, + NUM_SLOTS_PER_TM), + true); + + miniClusterResource.before(); + } + + @AfterClass + public static void tearDown() throws Exception { + miniClusterResource.after(); + + zkServer.stop(); + zkServer.close(); + } + + /** + * Verify that we don't start a job from scratch if we cannot restore any of the + * CompletedCheckpoints. + * + *

    Synchronization for the different steps and things we want to observe happens via + * latches in the test method and the methods of {@link CheckpointBlockingFunction}. + * + *

    The test follows these steps: + *

      + *
    1. Start job and block on a latch until we have done some checkpoints + *
    2. Block in the special function + *
    3. Move away the contents of the ZooKeeper HA directory to make restoring from + * checkpoints impossible + *
    4. Unblock the special function, which now induces a failure + *
    5. Make sure that the job does not recover successfully + *
    6. Move back the HA directory + *
    7. Make sure that the job recovers, we use a latch to ensure that the operator + * restored successfully + *
    + */ + @Test(timeout = 120_000L) + public void testRestoreBehaviourWithFaultyStateHandles() throws Exception { + CheckpointBlockingFunction.allowedInitializeCallsWithoutRestore.set(1); + CheckpointBlockingFunction.successfulRestores.set(0); + CheckpointBlockingFunction.illegalRestores.set(0); + CheckpointBlockingFunction.afterMessWithZooKeeper.set(false); + CheckpointBlockingFunction.failedAlready.set(false); + + waitForCheckpointLatch = new OneShotLatch(); + failInCheckpointLatch = new OneShotLatch(); + + ClusterClient clusterClient = miniClusterResource.getClusterClient(); + final Deadline deadline = Deadline.now().plus(TEST_TIMEOUT); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0)); + env.enableCheckpointing(10); // Flink doesn't allow lower than 10 ms + + File checkpointLocation = TEMPORARY_FOLDER.newFolder(); + env.setStateBackend((StateBackend) new FsStateBackend(checkpointLocation.toURI())); + + DataStreamSource source = env.addSource(new UnboundedSource()); + + source + .keyBy((str) -> str) + .map(new CheckpointBlockingFunction()); + + JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + JobID jobID = Preconditions.checkNotNull(jobGraph.getJobID()); + + clusterClient.setDetached(true); + clusterClient.submitJob(jobGraph, ZooKeeperHighAvailabilityITCase.class.getClassLoader()); + + // wait until we did some checkpoints + waitForCheckpointLatch.await(); + + // mess with the HA directory so that the job cannot restore + File movedCheckpointLocation = TEMPORARY_FOLDER.newFolder(); + int numCheckpoints = 0; + File[] files = haStorageDir.listFiles(); + assertNotNull(files); + for (File file : files) { + if (file.getName().startsWith("completedCheckpoint")) { + assertTrue(file.renameTo(new File(movedCheckpointLocation, file.getName()))); + numCheckpoints++; + } + } + // Note to future developers: This will break when we change Flink to not put the + // checkpoint metadata into the HA directory but instead rely on the fact that the + // actual checkpoint directory on DFS contains the checkpoint metadata. In this case, + // ZooKeeper will only contain a "handle" (read: String) that points to the metadata + // in DFS. The likely solution will be that we have to go directly to ZooKeeper, find + // out where the checkpoint is stored and mess with that. + assertTrue(numCheckpoints > 0); + + failInCheckpointLatch.trigger(); + + // Ensure that we see at least one cycle where the job tries to restart and fails. + CompletableFuture jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> clusterClient.getJobStatus(jobID), + Time.milliseconds(1), + deadline, + (jobStatus) -> jobStatus == JobStatus.RESTARTING, + TestingUtils.defaultScheduledExecutor()); + assertEquals(JobStatus.RESTARTING, jobStatusFuture.get()); + + jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> clusterClient.getJobStatus(jobID), + Time.milliseconds(1), + deadline, + (jobStatus) -> jobStatus == JobStatus.FAILING, + TestingUtils.defaultScheduledExecutor()); + assertEquals(JobStatus.FAILING, jobStatusFuture.get()); + + // move back the HA directory so that the job can restore + CheckpointBlockingFunction.afterMessWithZooKeeper.set(true); + + files = movedCheckpointLocation.listFiles(); + assertNotNull(files); + for (File file : files) { + if (file.getName().startsWith("completedCheckpoint")) { + assertTrue(file.renameTo(new File(haStorageDir, file.getName()))); + } + } + + // now the job should be able to go to RUNNING again and then eventually to FINISHED, + // which it only does if it could successfully restore + jobStatusFuture = FutureUtils.retrySuccesfulWithDelay( + () -> clusterClient.getJobStatus(jobID), + Time.milliseconds(50), + deadline, + (jobStatus) -> jobStatus == JobStatus.FINISHED, + TestingUtils.defaultScheduledExecutor()); + assertEquals(JobStatus.FINISHED, jobStatusFuture.get()); + + assertThat("We saw illegal restores.", CheckpointBlockingFunction.illegalRestores.get(), is(0)); + } + + private static class UnboundedSource implements SourceFunction { + private volatile boolean running = true; + + @Override + public void run(SourceContext ctx) throws Exception { + while (running && !CheckpointBlockingFunction.afterMessWithZooKeeper.get()) { + ctx.collect("hello"); + // don't overdo it ... ;-) + Thread.sleep(50); + } + } + + @Override + public void cancel() { + running = false; + } + } + + private static class CheckpointBlockingFunction + extends RichMapFunction + implements CheckpointedFunction { + + // verify that we only call initializeState() + // once with isRestored() == false. All other invocations must have isRestored() == true. This + // verifies that we don't restart a job from scratch in case the CompletedCheckpoints can't + // be read. + static AtomicInteger allowedInitializeCallsWithoutRestore = new AtomicInteger(1); + + // we count when we see restores that are not allowed. We only + // allow restores once we messed with the HA directory and moved it back again + static AtomicInteger illegalRestores = new AtomicInteger(0); + static AtomicInteger successfulRestores = new AtomicInteger(0); + + // whether we are after the phase where we messed with the ZooKeeper HA directory, i.e. + // whether it's now ok for a restore to happen + static AtomicBoolean afterMessWithZooKeeper = new AtomicBoolean(false); + + static AtomicBoolean failedAlready = new AtomicBoolean(false); + + // also have some state to write to the checkpoint + private final ValueStateDescriptor stateDescriptor = + new ValueStateDescriptor<>("state", StringSerializer.INSTANCE); + + @Override + public String map(String value) throws Exception { + getRuntimeContext().getState(stateDescriptor).update("42"); + return value; + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (context.getCheckpointId() > 5) { + waitForCheckpointLatch.trigger(); + failInCheckpointLatch.await(); + if (!failedAlready.getAndSet(true)) { + throw new RuntimeException("Failing on purpose."); + } + } + } + + @Override + public void initializeState(FunctionInitializationContext context) { + if (!context.isRestored()) { + int updatedValue = allowedInitializeCallsWithoutRestore.decrementAndGet(); + if (updatedValue < 0) { + illegalRestores.getAndIncrement(); + throw new RuntimeException("We are not allowed any more restores."); + } + } else { + if (!afterMessWithZooKeeper.get()) { + illegalRestores.getAndIncrement(); + } else if (successfulRestores.getAndIncrement() > 0) { + // already saw the one allowed successful restore + illegalRestores.getAndIncrement(); + } + } + } + } +} From cdbd5e1cc563d18a8eace790bd39764bcaa5ce5d Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 12 Mar 2018 11:23:40 +0100 Subject: [PATCH 117/268] [FLINK-8922] Revert "[FLINK-8859][checkpointing] RocksDB backend should pass WriteOption to Rocks.put() when restoring" We need to revert FLINK-8859 because it causes problems with RocksDB that make our automated tests fail on Travis. The change looks actually good and it is currently unclear why this can introduce such a problem. This might also be a Rocks in RocksDB. Nevertheless, for the sake of a proper release testing, we should revert the change for now. (cherry picked from commit a389b43) --- .../contrib/streaming/state/RocksDBKeyedStateBackend.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 5444dee443851..8f95b1812d844 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -680,7 +680,7 @@ private void restoreKVStateData() throws IOException, RocksDBException { if (RocksDBFullSnapshotOperation.hasMetaDataFollowsFlag(key)) { //clear the signal bit in the key to make it ready for insertion again RocksDBFullSnapshotOperation.clearMetaDataFollowsFlag(key); - rocksDBKeyedStateBackend.db.put(handle, rocksDBKeyedStateBackend.writeOptions, key, value); + rocksDBKeyedStateBackend.db.put(handle, key, value); //TODO this could be aware of keyGroupPrefixBytes and write only one byte if possible kvStateId = RocksDBFullSnapshotOperation.END_OF_KEY_GROUP_MARK & compressedKgInputView.readShort(); @@ -690,7 +690,7 @@ private void restoreKVStateData() throws IOException, RocksDBException { handle = currentStateHandleKVStateColumnFamilies.get(kvStateId); } } else { - rocksDBKeyedStateBackend.db.put(handle, rocksDBKeyedStateBackend.writeOptions, key, value); + rocksDBKeyedStateBackend.db.put(handle, key, value); } } } @@ -1091,7 +1091,6 @@ private void restoreKeyGroupsShardWithTemporaryHelperInstance( if (stateBackend.keyGroupRange.contains(keyGroup)) { stateBackend.db.put(targetColumnFamilyHandle, - stateBackend.writeOptions, iterator.key(), iterator.value()); } From 7cf56bc51c47d8d7cd943f44d2dad83e50324d05 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 1 Mar 2018 16:26:21 +0100 Subject: [PATCH 118/268] [FLINK-8274] [table] Split generated methods for preventing compiler exceptions This closes #5613. This closes #5174. --- .../apache/flink/table/api/TableConfig.scala | 33 +++- .../flink/table/codegen/CodeGenerator.scala | 143 ++++++++++++++---- .../codegen/CollectorCodeGenerator.scala | 66 +++++--- .../table/codegen/FunctionCodeGenerator.scala | 96 +++++++----- .../codegen/InputFormatCodeGenerator.scala | 6 +- .../runtime/batch/table/CorrelateITCase.scala | 15 ++ .../table/runtime/stream/sql/SqlITCase.scala | 54 ++++++- .../table/runtime/utils/StreamTestData.scala | 6 + .../utils/StreamingWithStateTestBase.scala | 3 +- 9 files changed, 320 insertions(+), 102 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala index c78a022bec44a..51c9a37f15534 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala @@ -40,13 +40,19 @@ class TableConfig { /** * Defines the configuration of Calcite for Table API and SQL queries. */ - private var calciteConfig = CalciteConfig.DEFAULT + private var calciteConfig: CalciteConfig = CalciteConfig.DEFAULT /** * Defines the default context for decimal division calculation. * We use Scala's default MathContext.DECIMAL128. */ - private var decimalContext = MathContext.DECIMAL128 + private var decimalContext: MathContext = MathContext.DECIMAL128 + + /** + * Specifies a threshold where generated code will be split into sub-function calls. Java has a + * maximum method length of 64 KB. This setting allows for finer granularity if necessary. + */ + private var maxGeneratedCodeLength: Int = 64000 // just an estimate /** * Sets the timezone for date/time/timestamp conversions. @@ -59,12 +65,12 @@ class TableConfig { /** * Returns the timezone for date/time/timestamp conversions. */ - def getTimeZone = timeZone + def getTimeZone: TimeZone = timeZone /** * Returns the NULL check. If enabled, all fields need to be checked for NULL first. */ - def getNullCheck = nullCheck + def getNullCheck: Boolean = nullCheck /** * Sets the NULL check. If enabled, all fields need to be checked for NULL first. @@ -99,6 +105,25 @@ class TableConfig { def setDecimalContext(mathContext: MathContext): Unit = { this.decimalContext = mathContext } + + /** + * Returns the current threshold where generated code will be split into sub-function calls. + * Java has a maximum method length of 64 KB. This setting allows for finer granularity if + * necessary. Default is 64000. + */ + def getMaxGeneratedCodeLength: Int = maxGeneratedCodeLength + + /** + * Returns the current threshold where generated code will be split into sub-function calls. + * Java has a maximum method length of 64 KB. This setting allows for finer granularity if + * necessary. Default is 64000. + */ + def setMaxGeneratedCodeLength(maxGeneratedCodeLength: Int): Unit = { + if (maxGeneratedCodeLength <= 0) { + throw new IllegalArgumentException("Length must be greater than 0.") + } + this.maxGeneratedCodeLength = maxGeneratedCodeLength + } } object TableConfig { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala index e4064d6f1acd8..44885e3d72e2c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -109,31 +109,45 @@ abstract class CodeGenerator( // set of member statements that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableMemberStatements = mutable.LinkedHashSet[String]() + protected val reusableMemberStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() // set of constructor statements that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableInitStatements = mutable.LinkedHashSet[String]() + protected val reusableInitStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() // set of open statements for RichFunction that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableOpenStatements = mutable.LinkedHashSet[String]() + protected val reusableOpenStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() // set of close statements for RichFunction that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableCloseStatements = mutable.LinkedHashSet[String]() + protected val reusableCloseStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() - // set of statements that will be added only once per record + // set of statements that will be added only once per record; + // code should only update member variables because local variables are not accessible if + // the code needs to be split; // we use a LinkedHashSet to keep the insertion order - protected val reusablePerRecordStatements = mutable.LinkedHashSet[String]() + protected val reusablePerRecordStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() // map of initial input unboxing expressions that will be added only once // (inputTerm, index) -> expr - protected val reusableInputUnboxingExprs = mutable.Map[(String, Int), GeneratedExpression]() + protected val reusableInputUnboxingExprs: mutable.Map[(String, Int), GeneratedExpression] = + mutable.Map[(String, Int), GeneratedExpression]() // set of constructor statements that will be added only once // we use a LinkedHashSet to keep the insertion order - protected val reusableConstructorStatements = mutable.LinkedHashSet[(String, String)]() + protected val reusableConstructorStatements: mutable.LinkedHashSet[(String, String)] = + mutable.LinkedHashSet[(String, String)]() + + /** + * Flag that indicates that the generated code needed to be split into several methods. + */ + protected var hasCodeSplits: Boolean = false /** * @return code block of statements that need to be placed in the member area of the Function @@ -384,7 +398,7 @@ abstract class CodeGenerator( returnType match { case ri: RowTypeInfo => addReusableOutRecord(ri) - val resultSetters: String = boxedFieldExprs.zipWithIndex map { + val resultSetters = boxedFieldExprs.zipWithIndex map { case (fieldExpr, i) => if (nullCheck) { s""" @@ -403,13 +417,15 @@ abstract class CodeGenerator( |$outRecordTerm.setField($i, ${fieldExpr.resultTerm}); |""".stripMargin } - } mkString "\n" + } + + val code = generateCodeSplits(resultSetters) - GeneratedExpression(outRecordTerm, "false", resultSetters, returnType) + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) case pt: PojoTypeInfo[_] => addReusableOutRecord(pt) - val resultSetters: String = boxedFieldExprs.zip(resultFieldNames) map { + val resultSetters = boxedFieldExprs.zip(resultFieldNames) map { case (fieldExpr, fieldName) => val accessor = getFieldAccessor(pt.getTypeClass, fieldName) @@ -474,13 +490,15 @@ abstract class CodeGenerator( |""".stripMargin } } - } mkString "\n" + } + + val code = generateCodeSplits(resultSetters) - GeneratedExpression(outRecordTerm, "false", resultSetters, returnType) + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) case tup: TupleTypeInfo[_] => addReusableOutRecord(tup) - val resultSetters: String = boxedFieldExprs.zipWithIndex map { + val resultSetters = boxedFieldExprs.zipWithIndex map { case (fieldExpr, i) => val fieldName = "f" + i if (nullCheck) { @@ -500,11 +518,13 @@ abstract class CodeGenerator( |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; |""".stripMargin } - } mkString "\n" + } + + val code = generateCodeSplits(resultSetters) - GeneratedExpression(outRecordTerm, "false", resultSetters, returnType) + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) - case cc: CaseClassTypeInfo[_] => + case _: CaseClassTypeInfo[_] => val fieldCodes: String = boxedFieldExprs.map(_.code).mkString("\n") val constructorParams: String = boxedFieldExprs.map(_.resultTerm).mkString(", ") val resultTerm = newName(outRecordTerm) @@ -528,9 +548,10 @@ abstract class CodeGenerator( |$returnTypeTerm $resultTerm = new $returnTypeTerm($constructorParams); |""".stripMargin - GeneratedExpression(resultTerm, "false", resultCode, returnType) + // case classes are not splittable + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, returnType) - case t: TypeInformation[_] => + case _: TypeInformation[_] => val fieldExpr = boxedFieldExprs.head val nullCheckCode = if (nullCheck) { s""" @@ -547,7 +568,8 @@ abstract class CodeGenerator( |$nullCheckCode |""".stripMargin - GeneratedExpression(fieldExpr.resultTerm, "false", resultCode, returnType) + // other types are not splittable + GeneratedExpression(fieldExpr.resultTerm, fieldExpr.nullTerm, resultCode, returnType) case _ => throw new CodeGenException(s"Unsupported result type: $returnType") @@ -1024,6 +1046,55 @@ abstract class CodeGenerator( // generator helping methods // ---------------------------------------------------------------------------------------------- + private def generateCodeSplits(splits: Seq[String]): String = { + val totalLen = splits.map(_.length + 1).sum // 1 for a line break + + // split + if (totalLen > config.getMaxGeneratedCodeLength) { + + hasCodeSplits = true + + // add input unboxing to member area such that all split functions can access it + reusableInputUnboxingExprs.foreach { case (_, expr) => + + // declaration + val resultTypeTerm = primitiveTypeTermForTypeInfo(expr.resultType) + if (nullCheck) { + reusableMemberStatements.add(s"private boolean ${expr.nullTerm};") + } + reusableMemberStatements.add(s"private $resultTypeTerm ${expr.resultTerm};") + + // assignment + if (nullCheck) { + reusablePerRecordStatements.add(s"this.${expr.nullTerm} = ${expr.nullTerm};") + } + reusablePerRecordStatements.add(s"this.${expr.resultTerm} = ${expr.resultTerm};") + } + + // add split methods to the member area and return the code necessary to call those methods + val methodCalls = splits.map { split => + val methodName = newName(s"split") + + val method = + s""" + |private final void $methodName() throws Exception { + | $split + |} + |""".stripMargin + reusableMemberStatements.add(method) + + // create method call + s"$methodName();" + } + + methodCalls.mkString("\n") + } + // don't split + else { + splits.mkString("\n") + } + } + private def generateFieldAccess(refExpr: GeneratedExpression, index: Int): GeneratedExpression = { val fieldAccessExpr = generateFieldAccess( @@ -1644,9 +1715,13 @@ abstract class CodeGenerator( def addReusableTimestamp(): String = { val fieldTerm = s"timestamp" + // declaration + reusableMemberStatements.add(s"private long $fieldTerm;") + + // assignment val field = s""" - |final long $fieldTerm = java.lang.System.currentTimeMillis(); + |$fieldTerm = java.lang.System.currentTimeMillis(); |""".stripMargin reusablePerRecordStatements.add(field) fieldTerm @@ -1660,9 +1735,13 @@ abstract class CodeGenerator( val timestamp = addReusableTimestamp() + // declaration + reusableMemberStatements.add(s"private long $fieldTerm;") + + // assignment val field = s""" - |final long $fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset(timestamp); + |$fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset($timestamp); |""".stripMargin reusablePerRecordStatements.add(field) fieldTerm @@ -1676,10 +1755,14 @@ abstract class CodeGenerator( val timestamp = addReusableTimestamp() + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime() val field = s""" - |final int $fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |$fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY}); |if (time < 0) { | time += ${DateTimeUtils.MILLIS_PER_DAY}; |} @@ -1696,10 +1779,14 @@ abstract class CodeGenerator( val localtimestamp = addReusableLocalTimestamp() + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment // adopted from org.apache.calcite.runtime.SqlFunctions.localTime() val field = s""" - |final int $fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |$fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY}); |""".stripMargin reusablePerRecordStatements.add(field) fieldTerm @@ -1715,10 +1802,14 @@ abstract class CodeGenerator( val timestamp = addReusableTimestamp() val time = addReusableTime() + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate() val field = s""" - |final int $fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY}); + |$fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY}); |if ($time < 0) { | $fieldTerm -= 1; |} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala index 70f6638998a44..9fc76e329833f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CollectorCodeGenerator.scala @@ -61,38 +61,54 @@ class CollectorCodeGenerator( * @return instance of GeneratedCollector */ def generateTableFunctionCollector( - name: String, - bodyCode: String, - collectedType: TypeInformation[Any]) - : GeneratedCollector = { + name: String, + bodyCode: String, + collectedType: TypeInformation[Any]) + : GeneratedCollector = { val className = newName(name) val input1TypeClass = boxedTypeTermForTypeInfo(input1) val input2TypeClass = boxedTypeTermForTypeInfo(collectedType) - val funcCode = j""" - public class $className extends ${classOf[TableFunctionCollector[_]].getCanonicalName} { - - ${reuseMemberCode()} + // declaration in case of code splits + val recordMember = if (hasCodeSplits) { + s"private $input2TypeClass $input2Term;" + } else { + "" + } - public $className() throws Exception { - ${reuseInitCode()} - } + // assignment in case of code splits + val recordAssignment = if (hasCodeSplits) { + s"$input2Term" // use member + } else { + s"$input2TypeClass $input2Term" // local variable + } - @Override - public void collect(Object record) throws Exception { - super.collect(record); - $input1TypeClass $input1Term = ($input1TypeClass) getInput(); - $input2TypeClass $input2Term = ($input2TypeClass) record; - ${reuseInputUnboxingCode()} - $bodyCode - } - - @Override - public void close() { - } - } - """.stripMargin + val funcCode = j""" + |public class $className extends ${classOf[TableFunctionCollector[_]].getCanonicalName} { + | + | $recordMember + | ${reuseMemberCode()} + | + | public $className() throws Exception { + | ${reuseInitCode()} + | } + | + | @Override + | public void collect(Object record) throws Exception { + | super.collect(record); + | $input1TypeClass $input1Term = ($input1TypeClass) getInput(); + | $recordAssignment = ($input2TypeClass) record; + | ${reuseInputUnboxingCode()} + | ${reusePerRecordCode()} + | $bodyCode + | } + | + | @Override + | public void close() { + | } + |} + |""".stripMargin GeneratedCollector(className, funcCode) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala index 2bd2fe7ef7ef9..8ac18cdda46e1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/FunctionCodeGenerator.scala @@ -24,6 +24,7 @@ import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.TableConfig import org.apache.flink.table.codegen.CodeGenUtils.{boxedTypeTermForTypeInfo, newName} import org.apache.flink.table.codegen.Indenter.toISC +import org.apache.flink.util.Collector /** * A code generator for generating Flink [[org.apache.flink.api.common.functions.Function]]s. @@ -85,22 +86,23 @@ class FunctionCodeGenerator( * @return instance of GeneratedFunction */ def generateFunction[F <: Function, T <: Any]( - name: String, - clazz: Class[F], - bodyCode: String, - returnType: TypeInformation[T]) - : GeneratedFunction[F, T] = { + name: String, + clazz: Class[F], + bodyCode: String, + returnType: TypeInformation[T]) + : GeneratedFunction[F, T] = { val funcName = newName(name) + val collectorTypeTerm = classOf[Collector[Any]].getCanonicalName // Janino does not support generics, that's why we need // manual casting here - val samHeader = + val (functionClass, signature, inputStatements) = // FlatMapFunction if (clazz == classOf[FlatMapFunction[_, _]]) { val baseClass = classOf[RichFlatMapFunction[_, _]] val inputTypeTerm = boxedTypeTermForTypeInfo(input1) (baseClass, - s"void flatMap(Object _in1, org.apache.flink.util.Collector $collectorTerm)", + s"void flatMap(Object _in1, $collectorTypeTerm $collectorTerm)", List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;")) } @@ -120,7 +122,7 @@ class FunctionCodeGenerator( val inputTypeTerm2 = boxedTypeTermForTypeInfo(input2.getOrElse( throw new CodeGenException("Input 2 for FlatJoinFunction should not be null"))) (baseClass, - s"void join(Object _in1, Object _in2, org.apache.flink.util.Collector $collectorTerm)", + s"void join(Object _in1, Object _in2, $collectorTypeTerm $collectorTerm)", List(s"$inputTypeTerm1 $input1Term = ($inputTypeTerm1) _in1;", s"$inputTypeTerm2 $input2Term = ($inputTypeTerm2) _in2;")) } @@ -141,11 +143,22 @@ class FunctionCodeGenerator( else if (clazz == classOf[ProcessFunction[_, _]]) { val baseClass = classOf[ProcessFunction[_, _]] val inputTypeTerm = boxedTypeTermForTypeInfo(input1) + val contextTypeTerm = classOf[ProcessFunction[Any, Any]#Context].getCanonicalName + + // make context accessible also for split code + val globalContext = if (hasCodeSplits) { + // declaration + reusableMemberStatements.add(s"private $contextTypeTerm $contextTerm;") + // assignment + List(s"this.$contextTerm = $contextTerm;") + } else { + Nil + } + (baseClass, - s"void processElement(Object _in1, " + - s"org.apache.flink.streaming.api.functions.ProcessFunction.Context $contextTerm," + - s"org.apache.flink.util.Collector $collectorTerm)", - List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;")) + s"void processElement(Object _in1, $contextTypeTerm $contextTerm, " + + s"$collectorTypeTerm $collectorTerm)", + List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;") ++ globalContext) } else { // TODO more functions @@ -153,36 +166,35 @@ class FunctionCodeGenerator( } val funcCode = j""" - public class $funcName - extends ${samHeader._1.getCanonicalName} { - - ${reuseMemberCode()} - - public $funcName() throws Exception { - ${reuseInitCode()} - } - - ${reuseConstructorCode(funcName)} - - @Override - public void open(${classOf[Configuration].getCanonicalName} parameters) throws Exception { - ${reuseOpenCode()} - } - - @Override - public ${samHeader._2} throws Exception { - ${samHeader._3.mkString("\n")} - ${reusePerRecordCode()} - ${reuseInputUnboxingCode()} - $bodyCode - } - - @Override - public void close() throws Exception { - ${reuseCloseCode()} - } - } - """.stripMargin + |public class $funcName extends ${functionClass.getCanonicalName} { + | + | ${reuseMemberCode()} + | + | public $funcName() throws Exception { + | ${reuseInitCode()} + | } + | + | ${reuseConstructorCode(funcName)} + | + | @Override + | public void open(${classOf[Configuration].getCanonicalName} parameters) throws Exception { + | ${reuseOpenCode()} + | } + | + | @Override + | public $signature throws Exception { + | ${inputStatements.mkString("\n")} + | ${reuseInputUnboxingCode()} + | ${reusePerRecordCode()} + | $bodyCode + | } + | + | @Override + | public void close() throws Exception { + | ${reuseCloseCode()} + | } + |} + |""".stripMargin GeneratedFunction(funcName, returnType, funcCode) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala index 6d6e1b675b063..30d33005b2ea3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/InputFormatCodeGenerator.scala @@ -71,12 +71,16 @@ class InputFormatCodeGenerator( } @Override - public Object nextRecord(Object reuse) { + public Object nextRecord(Object reuse) throws java.io.IOException { switch (nextIdx) { ${records.zipWithIndex.map { case (r, i) => s""" |case $i: + |try { | $r + |} catch (Exception e) { + | throw new java.io.IOException(e); + |} |break; """.stripMargin }.mkString("\n")} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala index 828a9e2654a62..b385015102ce3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CorrelateITCase.scala @@ -81,6 +81,21 @@ class CorrelateITCase( TestBaseUtils.compareResultAsText(results.asJava, expected) } + @Test + def testLeftOuterJoinWithSplit(): Unit = { + val env = ExecutionEnvironment.getExecutionEnvironment + val tableEnv = TableEnvironment.getTableEnvironment(env, config) + tableEnv.getConfig.setMaxGeneratedCodeLength(1) // split every field + val in = testData(env).toTable(tableEnv).as('a, 'b, 'c) + + val func2 = new TableFunc2 + val result = in.leftOuterJoin(func2('c) as ('s, 'l)).select('c, 's, 'l).toDataSet[Row] + val results = result.collect() + val expected = "Jack#22,Jack,4\n" + "Jack#22,22,2\n" + "John#19,John,4\n" + + "John#19,19,2\n" + "Anna#44,Anna,4\n" + "Anna#44,44,2\n" + "nosharp,null,null" + TestBaseUtils.compareResultAsText(results.asJava, expected) + } + /** * Common join predicates are temporarily forbidden (see FLINK-7865). */ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala index 1e2cf9c602183..b7950b7d9929e 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/SqlITCase.scala @@ -173,7 +173,7 @@ class SqlITCase extends StreamingWithStateTestBase { val t = StreamTestData.get3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c) tEnv.registerTable("MyTable", t) - val result = tEnv.sql(sqlQuery).toRetractStream[Row] + val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] result.addSink(new StreamITCase.RetractingSink).setParallelism(1) env.execute() @@ -208,7 +208,7 @@ class SqlITCase extends StreamingWithStateTestBase { tEnv.registerTable("MyTable", env.fromCollection(data).toTable(tEnv).as('a, 'b, 'c)) - val result = tEnv.sql(sqlQuery).toRetractStream[Row] + val result = tEnv.sqlQuery(sqlQuery).toRetractStream[Row] result.addSink(new StreamITCase.RetractingSink).setParallelism(1) env.execute() @@ -261,6 +261,27 @@ class SqlITCase extends StreamingWithStateTestBase { assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + @Test + def testSelectExpressionWithSplitFromTable(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + tEnv.getConfig.setMaxGeneratedCodeLength(1) // split every field + StreamITCase.clear + + val sqlQuery = "SELECT a * 2, b - 1 FROM MyTable" + + val t = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("MyTable", t) + + val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + val expected = List("2,0", "4,1", "6,1") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) + } + /** test filtering with registered table **/ @Test def testSimpleFilter(): Unit = { @@ -580,7 +601,7 @@ class SqlITCase extends StreamingWithStateTestBase { tEnv.registerTable("T1", t1) - val result = tEnv.sql(sqlQuery).toAppendStream[Row] + val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) env.execute() @@ -638,6 +659,33 @@ class SqlITCase extends StreamingWithStateTestBase { "3,3300") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } + + @Test + def testVeryBigQuery(): Unit = { + + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val t = StreamTestData.getSingletonDataStream(env).toTable(tEnv).as('a, 'b, 'c) + tEnv.registerTable("MyTable", t) + + val sqlQuery = new StringBuilder + sqlQuery.append("SELECT ") + val expected = new StringBuilder + for (i <- 0 until 500) { + sqlQuery.append(s"a + b + $i, ") + expected.append((1 + 42L + i).toString + ",") + } + sqlQuery.append("c FROM MyTable") + expected.append("Hi") + + val result = tEnv.sqlQuery(sqlQuery.toString()).toAppendStream[Row] + result.addSink(new StreamITCase.StringSink[Row]) + env.execute() + + assertEquals(List(expected.toString()), StreamITCase.testResults.sorted) + } } object SqlITCase { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala index 58d3c635a66bd..ef98791716afa 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamTestData.scala @@ -25,6 +25,12 @@ import scala.collection.mutable object StreamTestData { + def getSingletonDataStream(env: StreamExecutionEnvironment): DataStream[(Int, Long, String)] = { + val data = new mutable.MutableList[(Int, Long, String)] + data.+=((1, 42L, "Hi")) + env.fromCollection(data) + } + def getSmall3TupleDataStream(env: StreamExecutionEnvironment): DataStream[(Int, Long, String)] = { val data = new mutable.MutableList[(Int, Long, String)] data.+=((1, 1L, "Hi")) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala index 5cfab4aabac6d..b3eeb59bce5f2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithStateTestBase.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime.utils import org.apache.flink.contrib.streaming.state.RocksDBStateBackend +import org.apache.flink.runtime.state.StateBackend import org.apache.flink.test.util.AbstractTestBase import org.junit.Rule import org.junit.rules.TemporaryFolder @@ -29,7 +30,7 @@ class StreamingWithStateTestBase extends AbstractTestBase { @Rule def tempFolder: TemporaryFolder = _tempFolder - def getStateBackend: RocksDBStateBackend = { + def getStateBackend: StateBackend = { val dbPath = tempFolder.newFolder().getAbsolutePath val checkpointPath = tempFolder.newFolder().toURI.toString val backend = new RocksDBStateBackend(checkpointPath) From 2da0816cba79e797277cbf271fabb1d207738914 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 12 Mar 2018 12:45:37 +0100 Subject: [PATCH 119/268] [hotfix][RAT] Add serializer snapshot to exclusions --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index 887fb96e95cec..02ca63b1b2514 100644 --- a/pom.xml +++ b/pom.xml @@ -1024,6 +1024,7 @@ under the License. **/src/test/resources/*-savepoint flink-core/src/test/resources/serialized-kryo-serializer-1.3 flink-core/src/test/resources/type-without-avro-serialized-using-kryo + flink-formats/flink-avro/src/test/resources/flink-1.4-serializer-java-serialized flink-formats/flink-avro/src/test/resources/testdata.avro flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/generated/*.java From ccdfdf5dc9b19de68839be28e8e6adec542430b3 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 12 Mar 2018 15:08:36 +0100 Subject: [PATCH 120/268] [hotfix][tests] Strip CompletionExceptions in MiniClusterClient#guardWithSingleRetry --- .../org/apache/flink/client/program/MiniClusterClient.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index 7475071e23e6e..f0a7631023ea4 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.LeaderRetrievalUtils; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import javax.annotation.Nonnull; @@ -197,7 +198,10 @@ private static CompletableFuture guardWithSingleRetry(Supplier throwable instanceof FencingTokenException || throwable instanceof AkkaRpcException, + throwable -> { + Throwable actualException = ExceptionUtils.stripCompletionException(throwable); + return actualException instanceof FencingTokenException || actualException instanceof AkkaRpcException; + }, executor); } } From ea53eec0ff14ec55ae5f017cda074ed61870f84d Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 12 Mar 2018 16:28:14 +0100 Subject: [PATCH 121/268] [hotfix] Don't mark Table API & SQL as 'beta' anymore --- docs/dev/table/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/dev/table/index.md b/docs/dev/table/index.md index 5845c95099cbb..34e94684ecf3c 100644 --- a/docs/dev/table/index.md +++ b/docs/dev/table/index.md @@ -2,7 +2,7 @@ title: "Table API & SQL" nav-id: tableapi nav-parent_id: dev -is_beta: true +is_beta: false nav-show_overview: true nav-pos: 35 --- From 3a3caac9ff3b27fe9ad5b9868eba8e0ec44fdb9c Mon Sep 17 00:00:00 2001 From: mingleiZhang Date: Wed, 7 Mar 2018 10:36:52 +0800 Subject: [PATCH 122/268] [FLINK-8687] [sql-client] Make MaterializedCollectStreamResult#retrievePage to have resultLock This closes #5647. --- .../gateway/local/MaterializedCollectStreamResult.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedCollectStreamResult.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedCollectStreamResult.java index bd7f08ee0513f..7935da63e0bf8 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedCollectStreamResult.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedCollectStreamResult.java @@ -86,11 +86,13 @@ else if (!isRetrieving()) { @Override public List retrievePage(int page) { - if (page <= 0 || page > pageCount) { - throw new SqlExecutionException("Invalid page '" + page + "'."); - } + synchronized (resultLock) { + if (page <= 0 || page > pageCount) { + throw new SqlExecutionException("Invalid page '" + page + "'."); + } - return snapshot.subList(pageSize * (page - 1), Math.min(snapshot.size(), pageSize * page)); + return snapshot.subList(pageSize * (page - 1), Math.min(snapshot.size(), pageSize * page)); + } } // -------------------------------------------------------------------------------------------- From 84a202f15b9b0100c9817928fc77266010e7b666 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 12 Mar 2018 18:38:03 +0100 Subject: [PATCH 123/268] [hotfix] [yarn] Improve logging of container resources --- .../main/java/org/apache/flink/yarn/YarnResourceManager.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 46ef81bed1707..af789baf7e95a 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -438,7 +438,10 @@ private void requestYarnContainer(Resource resource, Priority priority) { resourceManagerClient.setHeartbeatInterval(FAST_YARN_HEARTBEAT_INTERVAL_MS); numPendingContainerRequests++; - log.info("Requesting new TaskManager container pending requests: {}", numPendingContainerRequests); + + log.info("Requesting new TaskExecutor container with resources {}. Number pending requests {}.", + resource, + numPendingContainerRequests); } private ContainerLaunchContext createTaskExecutorLaunchContext(Resource resource, String containerId, String host) From 69377cdc32d1b91b5dd1850afa550eaff808e7c8 Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Mon, 12 Mar 2018 15:04:57 +0100 Subject: [PATCH 124/268] [FLINK-8927][checkpointing] Eagerly release the checkpoint object in RocksDB incremental snapshots This closes #5682. (cherry picked from commit 3debf47) --- .../contrib/streaming/state/RocksDBKeyedStateBackend.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 8f95b1812d844..6a661210ec703 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -2318,8 +2318,9 @@ void takeSnapshot() throws Exception { } // create hard links of living files in the snapshot path - Checkpoint checkpoint = Checkpoint.create(stateBackend.db); - checkpoint.createCheckpoint(localBackupDirectory.getDirectory().getPath()); + try (Checkpoint checkpoint = Checkpoint.create(stateBackend.db)) { + checkpoint.createCheckpoint(localBackupDirectory.getDirectory().getPath()); + } } @Nonnull From afac526c5f54f7c4c964a344fb734e0cfa5b6f85 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 12 Mar 2018 18:04:38 +0100 Subject: [PATCH 125/268] [FLINK-8783] [tests] Harden SlotPoolRpcTest Wait for releasing of timed out pending slot requests before checking the number of pending slots requests. This closes #5684. --- .../runtime/jobmaster/slotpool/SlotPoolRpcTest.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java index cc837bc049652..4c736e8ba0284 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java @@ -195,6 +195,9 @@ public void testSlotAllocationTimeout() throws Exception { pool.start(JobMasterId.generate(), "foobar"); SlotPoolGateway slotPoolGateway = pool.getSelfGateway(SlotPoolGateway.class); + final CompletableFuture slotRequestTimeoutFuture = new CompletableFuture<>(); + pool.setTimeoutPendingSlotRequestConsumer(slotRequestTimeoutFuture::complete); + ResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); pool.connectToResourceManager(resourceManagerGateway); @@ -213,6 +216,9 @@ public void testSlotAllocationTimeout() throws Exception { assertTrue(ExceptionUtils.stripExecutionException(e) instanceof TimeoutException); } + // wait until we have timed out the slot request + slotRequestTimeoutFuture.get(); + assertEquals(0L, (long) pool.getNumberOfPendingRequests().get()); } finally { RpcUtils.terminateRpcEndpoint(pool, timeout); @@ -243,6 +249,9 @@ public void testExtraSlotsAreKept() throws Exception { resourceManagerGateway.setRequestSlotConsumer( (SlotRequest slotRequest) -> allocationIdFuture.complete(slotRequest.getAllocationId())); + final CompletableFuture slotRequestTimeoutFuture = new CompletableFuture<>(); + pool.setTimeoutPendingSlotRequestConsumer(slotRequestTimeoutFuture::complete); + pool.connectToResourceManager(resourceManagerGateway); SlotRequestId requestId = new SlotRequestId(); @@ -260,6 +269,9 @@ public void testExtraSlotsAreKept() throws Exception { assertTrue(ExceptionUtils.stripExecutionException(e) instanceof TimeoutException); } + // wait until we have timed out the slot request + slotRequestTimeoutFuture.get(); + assertEquals(0L, (long) pool.getNumberOfPendingRequests().get()); AllocationID allocationId = allocationIdFuture.get(); From 9551c6fa91bc1cc12352c9fe8464b5baf868268e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 13 Mar 2018 08:13:44 +0100 Subject: [PATCH 126/268] [FLINK-8934] [flip6] Properly cancel slot requests of otherwisely fulfilled requests Cancel slot requests at the ResourceManager if they have been completed with a different allocation. This closes #5687. --- .../runtime/jobmaster/slotpool/SlotPool.java | 2 +- .../jobmaster/slotpool/SlotPoolTest.java | 26 ++++++++++++++----- 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index 8a2dd45ea7ef8..42264b5321989 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -691,7 +691,7 @@ private void requestSlotFromResourceManager( pendingRequest.getAllocatedSlotFuture().whenComplete( (AllocatedSlot allocatedSlot, Throwable throwable) -> { - if (throwable != null || allocationId.equals(allocatedSlot.getAllocationId())) { + if (throwable != null || !allocationId.equals(allocatedSlot.getAllocationId())) { // cancel the slot request if there is a failure or if the pending request has // been completed with another allocated slot resourceManagerGateway.cancelSlotRequest(allocationId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java index c529ceb3c1a00..c3819747595b8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java @@ -457,16 +457,21 @@ public void testSlotRequestCancellationUponFailingRequest() throws Exception { * Tests that unused offered slots are directly used to fulfill pending slot * requests. * - *

    See FLINK-8089 + * Moreover it tests that the old slot request is canceled + * + *

    See FLINK-8089, FLINK-8934 */ @Test public void testFulfillingSlotRequestsWithUnusedOfferedSlots() throws Exception { final SlotPool slotPool = new SlotPool(rpcService, jobId); - final CompletableFuture allocationIdFuture = new CompletableFuture<>(); + final ArrayBlockingQueue allocationIds = new ArrayBlockingQueue<>(2); resourceManagerGateway.setRequestSlotConsumer( - (SlotRequest slotRequest) -> allocationIdFuture.complete(slotRequest.getAllocationId())); + (SlotRequest slotRequest) -> allocationIds.offer(slotRequest.getAllocationId())); + + final ArrayBlockingQueue canceledAllocations = new ArrayBlockingQueue<>(2); + resourceManagerGateway.setCancelSlotConsumer(canceledAllocations::offer); final SlotRequestId slotRequestId1 = new SlotRequestId(); final SlotRequestId slotRequestId2 = new SlotRequestId(); @@ -487,7 +492,7 @@ public void testFulfillingSlotRequestsWithUnusedOfferedSlots() throws Exception timeout); // wait for the first slot request - final AllocationID allocationId = allocationIdFuture.get(); + final AllocationID allocationId1 = allocationIds.take(); CompletableFuture slotFuture2 = slotPoolGateway.allocateSlot( slotRequestId2, @@ -496,6 +501,9 @@ public void testFulfillingSlotRequestsWithUnusedOfferedSlots() throws Exception true, timeout); + // wait for the second slot request + final AllocationID allocationId2 = allocationIds.take(); + slotPoolGateway.releaseSlot(slotRequestId1, null, null); try { @@ -505,17 +513,21 @@ public void testFulfillingSlotRequestsWithUnusedOfferedSlots() throws Exception } catch (ExecutionException ee) { // expected assertTrue(ExceptionUtils.stripExecutionException(ee) instanceof FlinkException); - } - final SlotOffer slotOffer = new SlotOffer(allocationId, 0, ResourceProfile.UNKNOWN); + assertEquals(allocationId1, canceledAllocations.take()); + + final SlotOffer slotOffer = new SlotOffer(allocationId1, 0, ResourceProfile.UNKNOWN); slotPoolGateway.registerTaskManager(taskManagerLocation.getResourceID()).get(); assertTrue(slotPoolGateway.offerSlot(taskManagerLocation, taskManagerGateway, slotOffer).get()); // the slot offer should fulfill the second slot request - assertEquals(allocationId, slotFuture2.get().getAllocationId()); + assertEquals(allocationId1, slotFuture2.get().getAllocationId()); + + // check that the second slot allocation has been canceled + assertEquals(allocationId2, canceledAllocations.take()); } finally { RpcUtils.terminateRpcEndpoint(slotPool, timeout); } From 3d0abd2824619cbf82ea4e91f74d859391a09d58 Mon Sep 17 00:00:00 2001 From: zjureel Date: Thu, 7 Sep 2017 10:39:39 +0800 Subject: [PATCH 127/268] [FLINK-7521] Add config option to set the content length limit of REST server and client --- .../flink/configuration/RestOptions.java | 15 ++++++++++++ .../apache/flink/runtime/rest/RestClient.java | 2 +- .../runtime/rest/RestClientConfiguration.java | 24 +++++++++++++++++-- .../runtime/rest/RestServerEndpoint.java | 5 +++- .../rest/RestServerEndpointConfiguration.java | 22 +++++++++++++++-- 5 files changed, 62 insertions(+), 6 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java index 888be08239888..61bb0853471bb 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java @@ -81,4 +81,19 @@ public class RestOptions { key("rest.connection-timeout") .defaultValue(15_000L) .withDescription("The maximum time in ms for the client to establish a TCP connection."); + + /** + * The max content length that the server will handle. + */ + public static final ConfigOption REST_SERVER_CONTENT_MAX_MB = + key("rest.server.content.max.mb") + .defaultValue(10); + + /** + * The max content length that the client will handle. + */ + public static final ConfigOption REST_CLIENT_CONTENT_MAX_MB = + key("rest.client.content.max.mb") + .defaultValue(1); + } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java index 3a0f6df70ac25..801119dfde40e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java @@ -103,7 +103,7 @@ protected void initChannel(SocketChannel socketChannel) throws Exception { socketChannel.pipeline() .addLast(new HttpClientCodec()) - .addLast(new HttpObjectAggregator(1024 * 1024)) + .addLast(new HttpObjectAggregator(configuration.getMaxContentLength())) .addLast(new ClientHandler()) .addLast(new PipelineErrorHandler(LOG)); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientConfiguration.java index 86578a2ce726f..782cb4e518e1f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientConfiguration.java @@ -39,9 +39,15 @@ public final class RestClientConfiguration { private final long connectionTimeout; - private RestClientConfiguration(@Nullable SSLEngine sslEngine, final long connectionTimeout) { + private final int maxContentLength; + + private RestClientConfiguration( + @Nullable final SSLEngine sslEngine, + final long connectionTimeout, + final int maxContentLength) { this.sslEngine = sslEngine; this.connectionTimeout = connectionTimeout; + this.maxContentLength = maxContentLength; } /** @@ -61,6 +67,15 @@ public long getConnectionTimeout() { return connectionTimeout; } + /** + * Returns the max content length that the REST client endpoint could handle. + * + * @return max content length that the REST client endpoint could handle + */ + public int getMaxContentLength() { + return maxContentLength; + } + /** * Creates and returns a new {@link RestClientConfiguration} from the given {@link Configuration}. * @@ -89,6 +104,11 @@ public static RestClientConfiguration fromConfiguration(Configuration config) th final long connectionTimeout = config.getLong(RestOptions.CONNECTION_TIMEOUT); - return new RestClientConfiguration(sslEngine, connectionTimeout); + int maxContentLength = config.getInteger(RestOptions.REST_CLIENT_CONTENT_MAX_MB) * 1024 * 1024; + if (maxContentLength <= 0) { + throw new ConfigurationException("Max content length for client must be a positive integer: " + maxContentLength); + } + + return new RestClientConfiguration(sslEngine, connectionTimeout, maxContentLength); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index f131ec186cbcf..42af4c6ee4ae9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -78,6 +78,7 @@ public abstract class RestServerEndpoint { private final int configuredPort; private final SSLEngine sslEngine; private final Path uploadDir; + private final int maxContentLength; private final CompletableFuture terminationFuture; @@ -96,6 +97,8 @@ public RestServerEndpoint(RestServerEndpointConfiguration configuration) throws this.uploadDir = configuration.getUploadDir(); createUploadDir(uploadDir, log); + this.maxContentLength = configuration.getMaxContentLength(); + terminationFuture = new CompletableFuture<>(); this.restAddress = null; @@ -156,7 +159,7 @@ protected void initChannel(SocketChannel ch) { ch.pipeline() .addLast(new HttpServerCodec()) .addLast(new FileUploadHandler(uploadDir)) - .addLast(new HttpObjectAggregator(MAX_REQUEST_SIZE_BYTES)) + .addLast(new HttpObjectAggregator(maxContentLength)) .addLast(handler.name(), handler) .addLast(new PipelineErrorHandler(log)); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java index c411b51cea75f..3685e2d656e50 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java @@ -51,11 +51,14 @@ public final class RestServerEndpointConfiguration { private final Path uploadDir; + private final int maxContentLength; + private RestServerEndpointConfiguration( @Nullable String restBindAddress, int restBindPort, @Nullable SSLEngine sslEngine, - final Path uploadDir) { + final Path uploadDir, + final int maxContentLength) { Preconditions.checkArgument(0 <= restBindPort && restBindPort < 65536, "The bing rest port " + restBindPort + " is out of range (0, 65536["); @@ -63,6 +66,7 @@ private RestServerEndpointConfiguration( this.restBindPort = restBindPort; this.sslEngine = sslEngine; this.uploadDir = requireNonNull(uploadDir); + this.maxContentLength = maxContentLength; } /** @@ -99,6 +103,15 @@ public Path getUploadDir() { return uploadDir; } + /** + * Returns the max content length that the REST server endpoint could handle. + * + * @return max content length that the REST server endpoint could handle + */ + public int getMaxContentLength() { + return maxContentLength; + } + /** * Creates and returns a new {@link RestServerEndpointConfiguration} from the given {@link Configuration}. * @@ -131,6 +144,11 @@ public static RestServerEndpointConfiguration fromConfiguration(Configuration co config.getString(WebOptions.UPLOAD_DIR, config.getString(WebOptions.TMP_DIR)), "flink-web-upload-" + UUID.randomUUID()); - return new RestServerEndpointConfiguration(address, port, sslEngine, uploadDir); + int maxContentLength = config.getInteger(RestOptions.REST_SERVER_CONTENT_MAX_MB) * 1024 * 1024; + if (maxContentLength <= 0) { + throw new ConfigurationException("Max content length for server must be a positive integer: " + maxContentLength); + } + + return new RestServerEndpointConfiguration(address, port, sslEngine, uploadDir, maxContentLength); } } From 8169ff5d471c2af342cd38d132c383262be798b2 Mon Sep 17 00:00:00 2001 From: gyao Date: Mon, 12 Mar 2018 15:44:27 +0100 Subject: [PATCH 128/268] [FLINK-7521][flip6] Remove RestServerEndpoint#MAX_REQUEST_SIZE_BYTES --- .../apache/flink/runtime/rest/RestServerEndpoint.java | 1 - .../runtime/rest/messages/job/JobSubmitRequestBody.java | 9 +-------- 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index 42af4c6ee4ae9..8b392508aeace 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -69,7 +69,6 @@ */ public abstract class RestServerEndpoint { - public static final int MAX_REQUEST_SIZE_BYTES = 1024 * 1024 * 10; protected final Logger log = LoggerFactory.getLogger(getClass()); private final Object lock = new Object(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitRequestBody.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitRequestBody.java index c472ff103d1c5..3f550f0baa775 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitRequestBody.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobSubmitRequestBody.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.rest.messages.job; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.rest.RestServerEndpoint; import org.apache.flink.runtime.rest.messages.RequestBody; import org.apache.flink.util.Preconditions; @@ -52,13 +51,7 @@ public JobSubmitRequestBody(JobGraph jobGraph) throws IOException { @JsonCreator public JobSubmitRequestBody( - @JsonProperty(FIELD_NAME_SERIALIZED_JOB_GRAPH) byte[] serializedJobGraph) { - - // check that job graph can be read completely by the HttpObjectAggregator on the server - // we subtract 1024 bytes to account for http headers and such. - if (serializedJobGraph.length > RestServerEndpoint.MAX_REQUEST_SIZE_BYTES - 1024) { - throw new IllegalArgumentException("Serialized job graph exceeded max request size."); - } + @JsonProperty(FIELD_NAME_SERIALIZED_JOB_GRAPH) byte[] serializedJobGraph) { this.serializedJobGraph = Preconditions.checkNotNull(serializedJobGraph); } From 23a0917ccd44689ee06379e7a402149940b0e79c Mon Sep 17 00:00:00 2001 From: gyao Date: Mon, 12 Mar 2018 23:16:25 +0100 Subject: [PATCH 129/268] [FLINK-7521][flip6] Return HTTP 413 if request limit is exceeded. Remove unnecessary PipelineErrorHandler from RestClient. Rename config keys for configuring request and response limits. Set response headers for all error responses. This closes #5685. --- .../flink/configuration/RestOptions.java | 18 +-- .../dispatcher/DispatcherRestEndpoint.java | 2 - .../rest/FlinkHttpObjectAggregator.java | 67 +++++++++++ .../apache/flink/runtime/rest/RestClient.java | 16 ++- .../runtime/rest/RestClientConfiguration.java | 8 +- .../runtime/rest/RestServerEndpoint.java | 10 +- .../rest/RestServerEndpointConfiguration.java | 34 +++++- .../rest/handler/PipelineErrorHandler.java | 16 ++- .../handler/RestHandlerConfiguration.java | 22 +--- .../runtime/rest/handler/RouterHandler.java | 13 +- .../rest/handler/util/HandlerUtils.java | 56 ++++++++- .../webmonitor/WebMonitorEndpoint.java | 2 - .../rest/RestServerEndpointITCase.java | 113 +++++++++++++++--- ...btaskCurrentAttemptDetailsHandlerTest.java | 2 +- ...ecutionAttemptAccumulatorsHandlerTest.java | 3 +- ...askExecutionAttemptDetailsHandlerTest.java | 2 +- 16 files changed, 304 insertions(+), 80 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/FlinkHttpObjectAggregator.java diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java index 61bb0853471bb..94d7977b72541 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java @@ -83,17 +83,19 @@ public class RestOptions { .withDescription("The maximum time in ms for the client to establish a TCP connection."); /** - * The max content length that the server will handle. + * The maximum content length that the server will handle. */ - public static final ConfigOption REST_SERVER_CONTENT_MAX_MB = - key("rest.server.content.max.mb") - .defaultValue(10); + public static final ConfigOption REST_SERVER_MAX_CONTENT_LENGTH = + key("rest.server.max-content-length") + .defaultValue(104_857_600) + .withDescription("The maximum content length in bytes that the server will handle."); /** - * The max content length that the client will handle. + * The maximum content length that the client will handle. */ - public static final ConfigOption REST_CLIENT_CONTENT_MAX_MB = - key("rest.client.content.max.mb") - .defaultValue(1); + public static final ConfigOption REST_CLIENT_MAX_CONTENT_LENGTH = + key("rest.client.max-content-length") + .defaultValue(104_857_600) + .withDescription("The maximum content length in bytes that the client will handle."); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java index 9df6deec49fca..45185528395d3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java @@ -44,7 +44,6 @@ import java.io.IOException; import java.nio.file.Path; import java.util.List; -import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -92,7 +91,6 @@ protected List> initiali // Add the Dispatcher specific handlers final Time timeout = restConfiguration.getTimeout(); - final Map responseHeaders = restConfiguration.getResponseHeaders(); BlobServerPortHandler blobServerPortHandler = new BlobServerPortHandler( restAddressFuture, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FlinkHttpObjectAggregator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FlinkHttpObjectAggregator.java new file mode 100644 index 0000000000000..4ee0256cbe20f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/FlinkHttpObjectAggregator.java @@ -0,0 +1,67 @@ +/* + * 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.rest; + +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.runtime.rest.handler.util.HandlerUtils; +import org.apache.flink.runtime.rest.messages.ErrorResponseBody; + +import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.TooLongFrameException; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObject; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +import javax.annotation.Nonnull; + +import java.util.List; +import java.util.Map; + +/** + * Same as {@link org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectDecoder} + * but returns HTTP 413 to the client if the payload exceeds {@link #maxContentLength}. + */ +public class FlinkHttpObjectAggregator extends org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectAggregator { + + private final Map responseHeaders; + + public FlinkHttpObjectAggregator(final int maxContentLength, @Nonnull final Map responseHeaders) { + super(maxContentLength); + this.responseHeaders = responseHeaders; + } + + @Override + protected void decode( + final ChannelHandlerContext ctx, + final HttpObject msg, + final List out) throws Exception { + + try { + super.decode(ctx, msg, out); + } catch (final TooLongFrameException e) { + HandlerUtils.sendErrorResponse( + ctx, + false, + new ErrorResponseBody(String.format( + e.getMessage() + " Try to raise [%s]", + RestOptions.REST_SERVER_MAX_CONTENT_LENGTH.key())), + HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE, + responseHeaders); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java index 801119dfde40e..6319634fe7587 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClient.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.runtime.rest.handler.PipelineErrorHandler; +import org.apache.flink.configuration.RestOptions; import org.apache.flink.runtime.rest.messages.ErrorResponseBody; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.rest.messages.MessageParameters; @@ -50,6 +50,7 @@ import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup; import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel; import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioSocketChannel; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.TooLongFrameException; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpRequest; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpRequest; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse; @@ -104,8 +105,7 @@ protected void initChannel(SocketChannel socketChannel) throws Exception { socketChannel.pipeline() .addLast(new HttpClientCodec()) .addLast(new HttpObjectAggregator(configuration.getMaxContentLength())) - .addLast(new ClientHandler()) - .addLast(new PipelineErrorHandler(LOG)); + .addLast(new ClientHandler()); } }; NioEventLoopGroup group = new NioEventLoopGroup(1, new DefaultThreadFactory("flink-rest-client-netty")); @@ -269,8 +269,14 @@ protected void channelRead0(ChannelHandlerContext ctx, Object msg) { } @Override - public void exceptionCaught(final ChannelHandlerContext ctx, final Throwable cause) throws Exception { - jsonFuture.completeExceptionally(cause); + public void exceptionCaught(final ChannelHandlerContext ctx, final Throwable cause) { + if (cause instanceof TooLongFrameException) { + jsonFuture.completeExceptionally(new TooLongFrameException(String.format( + cause.getMessage() + " Try to raise [%s]", + RestOptions.REST_CLIENT_MAX_CONTENT_LENGTH.key()))); + } else { + jsonFuture.completeExceptionally(cause); + } ctx.close(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientConfiguration.java index 782cb4e518e1f..17d4264565b64 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestClientConfiguration.java @@ -29,6 +29,8 @@ import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; +import static org.apache.flink.util.Preconditions.checkArgument; + /** * A configuration object for {@link RestClient}s. */ @@ -45,6 +47,7 @@ private RestClientConfiguration( @Nullable final SSLEngine sslEngine, final long connectionTimeout, final int maxContentLength) { + checkArgument(maxContentLength > 0, "maxContentLength must be positive, was: %d", maxContentLength); this.sslEngine = sslEngine; this.connectionTimeout = connectionTimeout; this.maxContentLength = maxContentLength; @@ -104,10 +107,7 @@ public static RestClientConfiguration fromConfiguration(Configuration config) th final long connectionTimeout = config.getLong(RestOptions.CONNECTION_TIMEOUT); - int maxContentLength = config.getInteger(RestOptions.REST_CLIENT_CONTENT_MAX_MB) * 1024 * 1024; - if (maxContentLength <= 0) { - throw new ConfigurationException("Max content length for client must be a positive integer: " + maxContentLength); - } + int maxContentLength = config.getInteger(RestOptions.REST_CLIENT_MAX_CONTENT_LENGTH); return new RestClientConfiguration(sslEngine, connectionTimeout, maxContentLength); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index 8b392508aeace..a3d48431f81e4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -37,7 +37,6 @@ import org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoopGroup; import org.apache.flink.shaded.netty4.io.netty.channel.socket.SocketChannel; import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectAggregator; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpServerCodec; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Handler; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Router; @@ -59,6 +58,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.TimeUnit; @@ -78,6 +78,7 @@ public abstract class RestServerEndpoint { private final SSLEngine sslEngine; private final Path uploadDir; private final int maxContentLength; + protected final Map responseHeaders; private final CompletableFuture terminationFuture; @@ -97,6 +98,7 @@ public RestServerEndpoint(RestServerEndpointConfiguration configuration) throws createUploadDir(uploadDir, log); this.maxContentLength = configuration.getMaxContentLength(); + this.responseHeaders = configuration.getResponseHeaders(); terminationFuture = new CompletableFuture<>(); @@ -148,7 +150,7 @@ public final void start() throws Exception { @Override protected void initChannel(SocketChannel ch) { - Handler handler = new RouterHandler(router); + Handler handler = new RouterHandler(router, responseHeaders); // SSL should be the first handler in the pipeline if (sslEngine != null) { @@ -158,9 +160,9 @@ protected void initChannel(SocketChannel ch) { ch.pipeline() .addLast(new HttpServerCodec()) .addLast(new FileUploadHandler(uploadDir)) - .addLast(new HttpObjectAggregator(maxContentLength)) + .addLast(new FlinkHttpObjectAggregator(maxContentLength, responseHeaders)) .addLast(handler.name(), handler) - .addLast(new PipelineErrorHandler(log)); + .addLast(new PipelineErrorHandler(log, responseHeaders)); } }; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java index 3685e2d656e50..35bd6ea15d77b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java @@ -26,12 +26,16 @@ import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.Preconditions; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders; + import javax.annotation.Nullable; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLEngine; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.Collections; +import java.util.Map; import java.util.UUID; import static java.util.Objects.requireNonNull; @@ -53,20 +57,24 @@ public final class RestServerEndpointConfiguration { private final int maxContentLength; + private final Map responseHeaders; + private RestServerEndpointConfiguration( @Nullable String restBindAddress, int restBindPort, @Nullable SSLEngine sslEngine, final Path uploadDir, - final int maxContentLength) { + final int maxContentLength, final Map responseHeaders) { Preconditions.checkArgument(0 <= restBindPort && restBindPort < 65536, "The bing rest port " + restBindPort + " is out of range (0, 65536["); + Preconditions.checkArgument(maxContentLength > 0, "maxContentLength must be positive, was: %d", maxContentLength); this.restBindAddress = restBindAddress; this.restBindPort = restBindPort; this.sslEngine = sslEngine; this.uploadDir = requireNonNull(uploadDir); this.maxContentLength = maxContentLength; + this.responseHeaders = requireNonNull(Collections.unmodifiableMap(responseHeaders)); } /** @@ -112,6 +120,13 @@ public int getMaxContentLength() { return maxContentLength; } + /** + * Response headers that should be added to every HTTP response. + */ + public Map getResponseHeaders() { + return responseHeaders; + } + /** * Creates and returns a new {@link RestServerEndpointConfiguration} from the given {@link Configuration}. * @@ -144,11 +159,18 @@ public static RestServerEndpointConfiguration fromConfiguration(Configuration co config.getString(WebOptions.UPLOAD_DIR, config.getString(WebOptions.TMP_DIR)), "flink-web-upload-" + UUID.randomUUID()); - int maxContentLength = config.getInteger(RestOptions.REST_SERVER_CONTENT_MAX_MB) * 1024 * 1024; - if (maxContentLength <= 0) { - throw new ConfigurationException("Max content length for server must be a positive integer: " + maxContentLength); - } + int maxContentLength = config.getInteger(RestOptions.REST_SERVER_MAX_CONTENT_LENGTH); + + final Map responseHeaders = Collections.singletonMap( + HttpHeaders.Names.ACCESS_CONTROL_ALLOW_ORIGIN, + config.getString(WebOptions.ACCESS_CONTROL_ALLOW_ORIGIN)); - return new RestServerEndpointConfiguration(address, port, sslEngine, uploadDir, maxContentLength); + return new RestServerEndpointConfiguration( + address, + port, + sslEngine, + uploadDir, + maxContentLength, + responseHeaders); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/PipelineErrorHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/PipelineErrorHandler.java index 046118a340163..a16b01fcbb736 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/PipelineErrorHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/PipelineErrorHandler.java @@ -30,6 +30,9 @@ import org.slf4j.Logger; import java.util.Collections; +import java.util.Map; + +import static java.util.Objects.requireNonNull; /** * This is the last handler in the pipeline. It logs all error messages. @@ -40,8 +43,11 @@ public class PipelineErrorHandler extends SimpleChannelInboundHandler responseHeaders; + + public PipelineErrorHandler(Logger logger, final Map responseHeaders) { + this.logger = requireNonNull(logger); + this.responseHeaders = requireNonNull(responseHeaders); } @Override @@ -59,5 +65,11 @@ protected void channelRead0(ChannelHandlerContext ctx, HttpRequest message) { @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { logger.warn("Unhandled exception", cause); + HandlerUtils.sendErrorResponse( + ctx, + false, + new ErrorResponseBody("Internal server error: " + cause.getMessage()), + HttpResponseStatus.INTERNAL_SERVER_ERROR, + responseHeaders); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java index acdd63c92f76f..f92946bd0f5f4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java @@ -23,11 +23,7 @@ import org.apache.flink.configuration.WebOptions; import org.apache.flink.util.Preconditions; -import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders; - import java.io.File; -import java.util.Collections; -import java.util.Map; import java.util.UUID; /** @@ -43,14 +39,11 @@ public class RestHandlerConfiguration { private final File tmpDir; - private final Map responseHeaders; - public RestHandlerConfiguration( long refreshInterval, int maxCheckpointStatisticCacheEntries, Time timeout, - File tmpDir, - Map responseHeaders) { + File tmpDir) { Preconditions.checkArgument(refreshInterval > 0L, "The refresh interval (ms) should be larger than 0."); this.refreshInterval = refreshInterval; @@ -58,8 +51,6 @@ public RestHandlerConfiguration( this.timeout = Preconditions.checkNotNull(timeout); this.tmpDir = Preconditions.checkNotNull(tmpDir); - - this.responseHeaders = Preconditions.checkNotNull(responseHeaders); } public long getRefreshInterval() { @@ -78,10 +69,6 @@ public File getTmpDir() { return tmpDir; } - public Map getResponseHeaders() { - return Collections.unmodifiableMap(responseHeaders); - } - public static RestHandlerConfiguration fromConfiguration(Configuration configuration) { final long refreshInterval = configuration.getLong(WebOptions.REFRESH_INTERVAL); @@ -92,15 +79,10 @@ public static RestHandlerConfiguration fromConfiguration(Configuration configura final String rootDir = "flink-web-" + UUID.randomUUID(); final File tmpDir = new File(configuration.getString(WebOptions.TMP_DIR), rootDir); - final Map responseHeaders = Collections.singletonMap( - HttpHeaders.Names.ACCESS_CONTROL_ALLOW_ORIGIN, - configuration.getString(WebOptions.ACCESS_CONTROL_ALLOW_ORIGIN)); - return new RestHandlerConfiguration( refreshInterval, maxCheckpointStatisticCacheEntries, timeout, - tmpDir, - responseHeaders); + tmpDir); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RouterHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RouterHandler.java index d1d08373ee6b4..fc02250ff805d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RouterHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RouterHandler.java @@ -27,20 +27,21 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Handler; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.router.Router; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import java.util.Map; -import java.util.Collections; +import static java.util.Objects.requireNonNull; /** * This class is an extension of {@link Handler} that replaces the standard error response to be identical with those * sent by the {@link AbstractRestHandler}. */ public class RouterHandler extends Handler { - private static final Logger LOG = LoggerFactory.getLogger(RouterHandler.class); - public RouterHandler(Router router) { + private final Map responseHeaders; + + public RouterHandler(Router router, final Map responseHeaders) { super(router); + this.responseHeaders = requireNonNull(responseHeaders); } @Override @@ -50,6 +51,6 @@ protected void respondNotFound(ChannelHandlerContext ctx, HttpRequest request) { request, new ErrorResponseBody("Not found."), HttpResponseStatus.NOT_FOUND, - Collections.emptyMap()); + responseHeaders); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java index a69f4aaf4576b..b407ada46e642 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/util/HandlerUtils.java @@ -112,6 +112,30 @@ public static void sendErrorResponse( HttpResponseStatus statusCode, Map headers) { + sendErrorResponse( + channelHandlerContext, + HttpHeaders.isKeepAlive(httpRequest), + errorMessage, + statusCode, + headers); + } + + /** + * Sends the given error response and status code to the given channel. + * + * @param channelHandlerContext identifying the open channel + * @param keepAlive If the connection should be kept alive. + * @param errorMessage which should be sent + * @param statusCode of the message to send + * @param headers additional header values + */ + public static void sendErrorResponse( + ChannelHandlerContext channelHandlerContext, + boolean keepAlive, + ErrorResponseBody errorMessage, + HttpResponseStatus statusCode, + Map headers) { + StringWriter sw = new StringWriter(); try { mapper.writeValue(sw, errorMessage); @@ -120,14 +144,14 @@ public static void sendErrorResponse( LOG.error("Internal server error. Could not map error response to JSON.", e); sendResponse( channelHandlerContext, - httpRequest, + keepAlive, "Internal server error. Could not map error response to JSON.", HttpResponseStatus.INTERNAL_SERVER_ERROR, headers); } sendResponse( channelHandlerContext, - httpRequest, + keepAlive, sw.toString(), statusCode, headers); @@ -148,6 +172,30 @@ public static void sendResponse( @Nonnull String message, @Nonnull HttpResponseStatus statusCode, @Nonnull Map headers) { + + sendResponse( + channelHandlerContext, + HttpHeaders.isKeepAlive(httpRequest), + message, + statusCode, + headers); + } + + /** + * Sends the given response and status code to the given channel. + * + * @param channelHandlerContext identifying the open channel + * @param keepAlive If the connection should be kept alive. + * @param message which should be sent + * @param statusCode of the message to send + * @param headers additional header values + */ + public static void sendResponse( + @Nonnull ChannelHandlerContext channelHandlerContext, + boolean keepAlive, + @Nonnull String message, + @Nonnull HttpResponseStatus statusCode, + @Nonnull Map headers) { HttpResponse response = new DefaultHttpResponse(HTTP_1_1, statusCode); response.headers().set(CONTENT_TYPE, RestConstants.REST_CONTENT_TYPE); @@ -156,7 +204,7 @@ public static void sendResponse( response.headers().set(headerEntry.getKey(), headerEntry.getValue()); } - if (HttpHeaders.isKeepAlive(httpRequest)) { + if (keepAlive) { response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE); } @@ -172,7 +220,7 @@ public static void sendResponse( ChannelFuture lastContentFuture = channelHandlerContext.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT); // close the connection, if no keep-alive is needed - if (!HttpHeaders.isKeepAlive(httpRequest)) { + if (!keepAlive) { lastContentFuture.addListener(ChannelFutureListener.CLOSE); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 10a3650ec2323..dfb2fc8591d97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -129,7 +129,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -199,7 +198,6 @@ protected List> initiali ArrayList> handlers = new ArrayList<>(30); final Time timeout = restConfiguration.getTimeout(); - final Map responseHeaders = restConfiguration.getResponseHeaders(); ClusterOverviewHandler clusterOverviewHandler = new ClusterOverviewHandler( restAddressFuture, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java index c9817ff19e437..32f3ec89cadd6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java @@ -51,10 +51,10 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.TooLongFrameException; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -81,7 +81,12 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -96,6 +101,7 @@ public class RestServerEndpointITCase extends TestLogger { private static final JobID QUERY_JOB_ID = new JobID(); private static final String JOB_ID_KEY = "jobid"; private static final Time timeout = Time.seconds(10L); + private static final int TEST_REST_MAX_CONTENT_LENGTH = 4096; @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -103,12 +109,15 @@ public class RestServerEndpointITCase extends TestLogger { private RestServerEndpoint serverEndpoint; private RestClient restClient; private TestUploadHandler testUploadHandler; + private InetSocketAddress serverAddress; @Before public void setup() throws Exception { Configuration config = new Configuration(); config.setInteger(RestOptions.REST_PORT, 0); config.setString(WebOptions.UPLOAD_DIR, temporaryFolder.newFolder().getCanonicalPath()); + config.setInteger(RestOptions.REST_SERVER_MAX_CONTENT_LENGTH, TEST_REST_MAX_CONTENT_LENGTH); + config.setInteger(RestOptions.REST_CLIENT_MAX_CONTENT_LENGTH, TEST_REST_MAX_CONTENT_LENGTH); RestServerEndpointConfiguration serverConfig = RestServerEndpointConfiguration.fromConfiguration(config); RestClientConfiguration clientConfig = RestClientConfiguration.fromConfiguration(config); @@ -133,6 +142,7 @@ public void setup() throws Exception { restClient = new TestRestClient(clientConfig); serverEndpoint.start(); + serverAddress = serverEndpoint.getServerAddress(); } @After @@ -161,7 +171,6 @@ public void testRequestInterleaving() throws Exception { // send first request and wait until the handler blocks CompletableFuture response1; - final InetSocketAddress serverAddress = serverEndpoint.getServerAddress(); synchronized (TestHandler.LOCK) { response1 = restClient.sendRequest( @@ -198,8 +207,6 @@ public void testRequestInterleaving() throws Exception { */ @Test public void testBadHandlerRequest() throws Exception { - final InetSocketAddress serverAddress = serverEndpoint.getServerAddress(); - final FaultyTestParameters parameters = new FaultyTestParameters(); parameters.faultyJobIDPathParameter.resolve(PATH_JOB_ID); @@ -215,11 +222,11 @@ public void testBadHandlerRequest() throws Exception { try { response.get(); - Assert.fail("The request should fail with a bad request return code."); + fail("The request should fail with a bad request return code."); } catch (ExecutionException ee) { Throwable t = ExceptionUtils.stripExecutionException(ee); - Assert.assertTrue(t instanceof RestClientException); + assertTrue(t instanceof RestClientException); RestClientException rce = (RestClientException) t; @@ -227,6 +234,50 @@ public void testBadHandlerRequest() throws Exception { } } + /** + * Tests that requests and responses larger than {@link #TEST_REST_MAX_CONTENT_LENGTH} + * are rejected by the server and client, respectively. + */ + @Test + public void testMaxContentLengthLimit() throws Exception { + final TestParameters parameters = new TestParameters(); + parameters.jobIDPathParameter.resolve(PATH_JOB_ID); + parameters.jobIDQueryParameter.resolve(Collections.singletonList(QUERY_JOB_ID)); + + CompletableFuture response; + response = restClient.sendRequest( + serverAddress.getHostName(), + serverAddress.getPort(), + new TestHeaders(), + parameters, + new TestRequest(2, createStringOfSize(TEST_REST_MAX_CONTENT_LENGTH))); + + try { + response.get(); + fail("Expected exception not thrown"); + } catch (final ExecutionException e) { + final Throwable throwable = ExceptionUtils.stripExecutionException(e); + assertThat(throwable, instanceOf(RestClientException.class)); + assertThat(throwable.getMessage(), containsString("Try to raise")); + } + + response = restClient.sendRequest( + serverAddress.getHostName(), + serverAddress.getPort(), + new TestHeaders(), + parameters, + new TestRequest(TestHandler.LARGE_RESPONSE_BODY_ID)); + + try { + response.get(); + fail("Expected exception not thrown"); + } catch (final ExecutionException e) { + final Throwable throwable = ExceptionUtils.stripExecutionException(e); + assertThat(throwable, instanceOf(TooLongFrameException.class)); + assertThat(throwable.getMessage(), containsString("Try to raise")); + } + } + /** * Tests that multipart/form-data uploads work correctly. * @@ -294,6 +345,14 @@ private static String generateMultiPartBoundary() { return Long.toHexString(System.currentTimeMillis()); } + private static String createStringOfSize(int size) { + StringBuilder sb = new StringBuilder(size); + for (int i = 0; i < size; i++) { + sb.append('a'); + } + return sb.toString(); + } + private static class TestRestServerEndpoint extends RestServerEndpoint { private final TestHandler testHandler; @@ -323,12 +382,14 @@ protected void startInternal() throws Exception {} private static class TestHandler extends AbstractRestHandler { - public static final Object LOCK = new Object(); + private static final Object LOCK = new Object(); + + private static final int LARGE_RESPONSE_BODY_ID = 3; TestHandler( - CompletableFuture localAddressFuture, - GatewayRetriever leaderRetriever, - Time timeout) { + CompletableFuture localAddressFuture, + GatewayRetriever leaderRetriever, + Time timeout) { super( localAddressFuture, leaderRetriever, @@ -342,7 +403,8 @@ protected CompletableFuture handleRequest(@Nonnull HandlerRequest< assertEquals(request.getPathParameter(JobIDPathParameter.class), PATH_JOB_ID); assertEquals(request.getQueryParameter(JobIDQueryParameter.class).get(0), QUERY_JOB_ID); - if (request.getRequestBody().id == 1) { + final int id = request.getRequestBody().id; + if (id == 1) { synchronized (LOCK) { try { LOCK.notifyAll(); @@ -350,8 +412,12 @@ protected CompletableFuture handleRequest(@Nonnull HandlerRequest< } catch (InterruptedException ignored) { } } + } else if (id == LARGE_RESPONSE_BODY_ID) { + return CompletableFuture.completedFuture(new TestResponse( + id, + createStringOfSize(TEST_REST_MAX_CONTENT_LENGTH))); } - return CompletableFuture.completedFuture(new TestResponse(request.getRequestBody().id)); + return CompletableFuture.completedFuture(new TestResponse(id)); } } @@ -365,18 +431,37 @@ private static class TestRestClient extends RestClient { private static class TestRequest implements RequestBody { public final int id; + public final String content; + + public TestRequest(int id) { + this(id, null); + } + @JsonCreator - public TestRequest(@JsonProperty("id") int id) { + public TestRequest( + @JsonProperty("id") int id, + @JsonProperty("content") final String content) { this.id = id; + this.content = content; } } private static class TestResponse implements ResponseBody { + public final int id; + public final String content; + + public TestResponse(int id) { + this(id, null); + } + @JsonCreator - public TestResponse(@JsonProperty("id") int id) { + public TestResponse( + @JsonProperty("id") int id, + @JsonProperty("content") String content) { this.id = id; + this.content = content; } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java index 997601fe99cf8..af8b995a3e66f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java @@ -126,7 +126,7 @@ public void testHandleRequest() throws Exception { CompletableFuture.completedFuture("127.0.0.1:9527"), () -> null, Time.milliseconds(100), - restHandlerConfiguration.getResponseHeaders(), + Collections.emptyMap(), SubtaskCurrentAttemptDetailsHeaders.getInstance(), new ExecutionGraphCache( restHandlerConfiguration.getTimeout(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java index 5f03c5572acd5..318541d288612 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java @@ -41,6 +41,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -62,7 +63,7 @@ public void testHandleRequest() throws Exception { CompletableFuture.completedFuture("127.0.0.1:9527"), () -> null, Time.milliseconds(100L), - restHandlerConfiguration.getResponseHeaders(), + Collections.emptyMap(), SubtaskExecutionAttemptAccumulatorsHeaders.getInstance(), new ExecutionGraphCache( restHandlerConfiguration.getTimeout(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java index d55ab775a1488..8e44c0e9731b0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java @@ -129,7 +129,7 @@ public void testHandleRequest() throws Exception { CompletableFuture.completedFuture("127.0.0.1:9527"), () -> null, Time.milliseconds(100L), - restHandlerConfiguration.getResponseHeaders(), + Collections.emptyMap(), SubtaskExecutionAttemptDetailsHeaders.getInstance(), new ExecutionGraphCache( restHandlerConfiguration.getTimeout(), From 2a29e739afc30f9cc37a5cb2abb84e010a390843 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 12 Mar 2018 14:56:32 +0100 Subject: [PATCH 130/268] [FLINK-8850] [sql-client] Add support for event-time in SQL Client This closes #5683. --- .../conf/sql-client-defaults.yaml | 32 ++++++++++++------ .../flink/table/client/config/Execution.java | 16 +++++++++ .../table/client/config/PropertyStrings.java | 6 ++++ .../flink/table/client/gateway/Executor.java | 3 +- .../gateway/local/ExecutionContext.java | 1 + .../client/gateway/local/LocalExecutor.java | 5 +-- .../client/gateway/local/ResultStore.java | 2 +- .../client/gateway/local/DependencyTest.java | 1 + .../gateway/local/LocalExecutorITCase.java | 1 + .../gateway/utils/TestTableSourceFactory.java | 33 +++++++++++++++++-- .../resources/test-sql-client-defaults.yaml | 1 + .../resources/test-sql-client-factory.yaml | 7 ++++ .../apache/flink/table/api/TableSchema.scala | 24 ++++++++++++-- 13 files changed, 113 insertions(+), 19 deletions(-) diff --git a/flink-libraries/flink-sql-client/conf/sql-client-defaults.yaml b/flink-libraries/flink-sql-client/conf/sql-client-defaults.yaml index 76ccd0c1292ef..35584222e22f9 100644 --- a/flink-libraries/flink-sql-client/conf/sql-client-defaults.yaml +++ b/flink-libraries/flink-sql-client/conf/sql-client-defaults.yaml @@ -41,12 +41,20 @@ sources: [] # empty list # Execution properties allow for changing the behavior of a table program. execution: - type: streaming # 'batch' or 'streaming' execution - result-mode: changelog # 'changelog' or 'table' presentation of results - parallelism: 1 # parallelism of the program - max-parallelism: 128 # maximum parallelism - min-idle-state-retention: 0 # minimum idle state retention in ms - max-idle-state-retention: 0 # maximum idle state retention in ms + # 'batch' or 'streaming' execution + type: streaming + # allow 'event-time' or only 'processing-time' in sources + time-characteristic: event-time + # 'changelog' or 'table' presentation of results + result-mode: changelog + # parallelism of the program + parallelism: 1 + # maximum parallelism + max-parallelism: 128 + # minimum idle state retention in ms + min-idle-state-retention: 0 + # maximum idle state retention in ms + max-idle-state-retention: 0 #============================================================================== # Deployment properties @@ -56,9 +64,13 @@ execution: # programs are submitted to. deployment: - type: standalone # only the 'standalone' deployment is supported - response-timeout: 5000 # general cluster communication timeout in ms - gateway-address: "" # (optional) address from cluster to gateway - gateway-port: 0 # (optional) port from cluster to gateway + # only the 'standalone' deployment is supported + type: standalone + # general cluster communication timeout in ms + response-timeout: 5000 + # (optional) address from cluster to gateway + gateway-address: "" + # (optional) port from cluster to gateway + gateway-port: 0 diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Execution.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Execution.java index 37d1a34e3d4cd..d84c35b1d2b33 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Execution.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Execution.java @@ -18,6 +18,8 @@ package org.apache.flink.table.client.config; +import org.apache.flink.streaming.api.TimeCharacteristic; + import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -54,6 +56,20 @@ public boolean isBatchExecution() { PropertyStrings.EXECUTION_TYPE_VALUE_BATCH); } + public TimeCharacteristic getTimeCharacteristic() { + final String s = properties.getOrDefault( + PropertyStrings.EXECUTION_TIME_CHARACTERISTIC, + PropertyStrings.EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME); + switch (s) { + case PropertyStrings.EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME: + return TimeCharacteristic.EventTime; + case PropertyStrings.EXECUTION_TIME_CHARACTERISTIC_VALUE_PROCESSING_TIME: + return TimeCharacteristic.ProcessingTime; + default: + return TimeCharacteristic.EventTime; + } + } + public long getMinStateRetention() { return Long.parseLong(properties.getOrDefault(PropertyStrings.EXECUTION_MIN_STATE_RETENTION, Long.toString(Long.MIN_VALUE))); } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/PropertyStrings.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/PropertyStrings.java index ba0759d3b0630..b7a3101dbcf5b 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/PropertyStrings.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/PropertyStrings.java @@ -35,6 +35,12 @@ private PropertyStrings() { public static final String EXECUTION_TYPE_VALUE_BATCH = "batch"; + public static final String EXECUTION_TIME_CHARACTERISTIC = "time-characteristic"; + + public static final String EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME = "event-time"; + + public static final String EXECUTION_TIME_CHARACTERISTIC_VALUE_PROCESSING_TIME = "processing-time"; + public static final String EXECUTION_MIN_STATE_RETENTION = "min-idle-state-retention"; public static final String EXECUTION_MAX_STATE_RETENTION = "max-idle-state-retention"; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java index 4a41222700e26..74e6a6b2dbb19 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/Executor.java @@ -46,7 +46,8 @@ public interface Executor { List listTables(SessionContext session) throws SqlExecutionException; /** - * Returns the schema of a table. Throws an exception if the table could not be found. + * Returns the schema of a table. Throws an exception if the table could not be found. The + * schema might contain time attribute types for helping the user during debugging a query. */ TableSchema getTableSchema(SessionContext session, String name) throws SqlExecutionException; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 15a3c129bb801..a013afcc0a424 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -149,6 +149,7 @@ private StreamExecutionEnvironment createStreamExecutionEnvironment() { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(mergedEnv.getExecution().getParallelism()); env.setMaxParallelism(mergedEnv.getExecution().getMaxParallelism()); + env.setStreamTimeCharacteristic(mergedEnv.getExecution().getTimeCharacteristic()); return env; } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index 35d7da9bbfd28..fa6c9d2fd26cd 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -228,6 +228,7 @@ public ResultDescriptor executeQuery(SessionContext session, String query) throw // create table here to fail quickly for wrong queries final Table table = createTable(context, query); + final TableSchema resultSchema = table.getSchema().withoutTimeAttributes(); // deployment final ClusterClient clusterClient = createDeployment(mergedEnv.getDeployment()); @@ -235,7 +236,7 @@ public ResultDescriptor executeQuery(SessionContext session, String query) throw // initialize result final DynamicResult result = resultStore.createResult( mergedEnv, - table.getSchema(), + resultSchema, context.getExecutionConfig()); // create job graph with jars @@ -275,7 +276,7 @@ public ResultDescriptor executeQuery(SessionContext session, String query) throw // start result retrieval result.startRetrieval(program); - return new ResultDescriptor(resultId, table.getSchema(), result.isMaterialized()); + return new ResultDescriptor(resultId, resultSchema, result.isMaterialized()); } @Override diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java index 1f3dc8484f0d9..19a440e22246b 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java @@ -55,7 +55,7 @@ public ResultStore(Configuration flinkConfig) { } /** - * Creates a result. Might start thread or opens sockets so every creates result must be closed. + * Creates a result. Might start threads or opens sockets so every created result must be closed. */ public DynamicResult createResult(Environment env, TableSchema schema, ExecutionConfig config) { if (!env.getExecution().isStreamingExecution()) { diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java index 715d2db5c39c2..40a1c2cf137c2 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java @@ -65,6 +65,7 @@ public void testTableSourceFactoryDiscovery() throws Exception { final TableSchema expected = TableSchema.builder() .field("IntegerField1", Types.INT()) .field("StringField1", Types.STRING()) + .field("rowtimeField", Types.SQL_TIMESTAMP()) .build(); assertEquals(expected, result); diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java index a2ae28108bfb9..45369784f563c 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java @@ -101,6 +101,7 @@ public void testGetSessionProperties() throws Exception { final Map expectedProperties = new HashMap<>(); expectedProperties.put("execution.type", "streaming"); + expectedProperties.put("execution.time-characteristic", "event-time"); expectedProperties.put("execution.parallelism", "1"); expectedProperties.put("execution.max-parallelism", "16"); expectedProperties.put("execution.max-idle-state-retention", "0"); diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java index 40a7e7bac6785..1b0a30e35615e 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/utils/TestTableSourceFactory.java @@ -25,6 +25,10 @@ import org.apache.flink.table.api.Types; import org.apache.flink.table.client.gateway.local.DependencyTest; import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.SchemaValidator; +import org.apache.flink.table.sources.DefinedProctimeAttribute; +import org.apache.flink.table.sources.DefinedRowtimeAttributes; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; import org.apache.flink.table.sources.StreamTableSource; import org.apache.flink.table.sources.TableSource; import org.apache.flink.table.sources.TableSourceFactory; @@ -34,8 +38,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_TIMESTAMPS_TYPE; +import static org.apache.flink.table.descriptors.RowtimeValidator.ROWTIME_WATERMARKS_TYPE; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; @@ -58,6 +65,8 @@ public List supportedProperties() { properties.add("connector.test-property"); properties.add(SCHEMA() + ".#." + SCHEMA_TYPE()); properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); + properties.add(SCHEMA() + ".#." + ROWTIME_TIMESTAMPS_TYPE()); + properties.add(SCHEMA() + ".#." + ROWTIME_WATERMARKS_TYPE()); return properties; } @@ -65,9 +74,13 @@ public List supportedProperties() { public TableSource create(Map properties) { final DescriptorProperties params = new DescriptorProperties(true); params.putProperties(properties); + final Optional proctime = SchemaValidator.deriveProctimeAttribute(params); + final List rowtime = SchemaValidator.deriveRowtimeAttributes(params); return new TestTableSource( params.getTableSchema(SCHEMA()), - properties.get("connector.test-property")); + properties.get("connector.test-property"), + proctime.orElse(null), + rowtime); } // -------------------------------------------------------------------------------------------- @@ -75,14 +88,18 @@ public TableSource create(Map properties) { /** * Test table source. */ - public static class TestTableSource implements StreamTableSource { + public static class TestTableSource implements StreamTableSource, DefinedRowtimeAttributes, DefinedProctimeAttribute { private final TableSchema schema; private final String property; + private final String proctime; + private final List rowtime; - public TestTableSource(TableSchema schema, String property) { + public TestTableSource(TableSchema schema, String property, String proctime, List rowtime) { this.schema = schema; this.property = property; + this.proctime = proctime; + this.rowtime = rowtime; } public String getProperty() { @@ -108,5 +125,15 @@ public TableSchema getTableSchema() { public String explainSource() { return "TestTableSource"; } + + @Override + public List getRowtimeAttributeDescriptors() { + return rowtime; + } + + @Override + public String getProctimeAttribute() { + return proctime; + } } } diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml index 9cbecb07903ab..5a598f15ab42b 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml @@ -63,6 +63,7 @@ sources: execution: type: streaming + time-characteristic: event-time parallelism: 1 max-parallelism: 16 min-idle-state-retention: 0 diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml index 1bb69e537f40a..daa1fd167b5f2 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml @@ -30,6 +30,13 @@ sources: type: INT - name: StringField1 type: VARCHAR + - name: rowtimeField + type: TIMESTAMP + rowtime: + timestamps: + type: from-source + watermarks: + type: from-source connector: type: "$VAR_0" $VAR_1: "$VAR_2" diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala index 6958b3d15da8a..6389b55b12509 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableSchema.scala @@ -23,6 +23,8 @@ import org.apache.flink.api.common.typeutils.CompositeType import _root_.scala.collection.mutable.ArrayBuffer import _root_.java.util.Objects +import org.apache.flink.table.calcite.FlinkTypeFactory + /** * A TableSchema represents a Table's structure. */ @@ -30,6 +32,8 @@ class TableSchema( private val columnNames: Array[String], private val columnTypes: Array[TypeInformation[_]]) { + private val columnNameToIndex: Map[String, Int] = columnNames.zipWithIndex.toMap + if (columnNames.length != columnTypes.length) { throw new TableException( s"Number of field names and field types must be equal.\n" + @@ -52,8 +56,6 @@ class TableSchema( s"List of all fields: ${columnNames.mkString("[", ", ", "]")}.") } - val columnNameToIndex: Map[String, Int] = columnNames.zipWithIndex.toMap - /** * Returns a deep copy of the TableSchema. */ @@ -115,6 +117,24 @@ class TableSchema( } } + /** + * Converts a table schema into a schema that represents the result that would be written + * into a table sink or operator outside of the Table & SQL API. Time attributes are replaced + * by proper TIMESTAMP data types. + * + * @return a table schema with no time attributes + */ + def withoutTimeAttributes: TableSchema = { + val converted = columnTypes.map { t => + if (FlinkTypeFactory.isTimeIndicatorType(t)) { + Types.SQL_TIMESTAMP + } else { + t + } + } + new TableSchema(columnNames, converted) + } + override def toString: String = { val builder = new StringBuilder builder.append("root\n") From ed04f9c48de9adbf794fcfa67c98ba266c69428f Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Fri, 2 Mar 2018 14:48:20 +0100 Subject: [PATCH 131/268] [FLINK-8832] [sql-client] Create a SQL Client Kafka 0.11 fat-jar This closes #5673. --- .../flink-connector-kafka-0.11/pom.xml | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/flink-connectors/flink-connector-kafka-0.11/pom.xml b/flink-connectors/flink-connector-kafka-0.11/pom.xml index 0fc1f134a9862..089e5a2a6070b 100644 --- a/flink-connectors/flink-connector-kafka-0.11/pom.xml +++ b/flink-connectors/flink-connector-kafka-0.11/pom.xml @@ -211,6 +211,55 @@ under the License. + + + + release + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + sql-jar + + + org.apache.kafka:* + org.apache.flink:flink-connector-kafka-base_${scala.binary.version} + org.apache.flink:flink-connector-kafka-0.9_${scala.binary.version} + org.apache.flink:flink-connector-kafka-0.10_${scala.binary.version} + + + + + *:* + + kafka/kafka-version.properties + + + + + + org.apache.kafka + org.apache.flink.kafka011.shaded.org.apache.kafka + + + + + + + + + + + From 3ad2489ac4cac39c6e3e82f9745e1f52ad0b0f5f Mon Sep 17 00:00:00 2001 From: yanghua Date: Mon, 12 Mar 2018 12:12:56 +0800 Subject: [PATCH 132/268] [FLINK-8916][REST] Write/read checkpointing mode enum in lower case This closes #5679. --- .../checkpoints/CheckpointConfigInfo.java | 46 ++++++++++++++++++- 1 file changed, 45 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointConfigInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointConfigInfo.java index 7a5d99fae370e..b0f6abf5215cd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointConfigInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointConfigInfo.java @@ -24,7 +24,16 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; - +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; import java.util.Objects; /** @@ -145,8 +154,43 @@ public int hashCode() { /** * Processing mode. */ + @JsonSerialize(using = ProcessingModeSerializer.class) + @JsonDeserialize(using = ProcessingModeDeserializer.class) public enum ProcessingMode { AT_LEAST_ONCE, EXACTLY_ONCE } + + /** + * JSON deserializer for {@link ProcessingMode}. + */ + public static class ProcessingModeSerializer extends StdSerializer { + + public ProcessingModeSerializer() { + super(ProcessingMode.class); + } + + @Override + public void serialize(ProcessingMode mode, JsonGenerator generator, SerializerProvider serializerProvider) + throws IOException { + generator.writeString(mode.name().toLowerCase()); + } + } + + /** + * Processing mode deserializer. + */ + public static class ProcessingModeDeserializer extends StdDeserializer { + + public ProcessingModeDeserializer() { + super(ProcessingMode.class); + } + + @Override + public ProcessingMode deserialize(JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + return ProcessingMode.valueOf(jsonParser.getValueAsString().toUpperCase()); + } + } + } From 74e6441719d77c1c9bab49a6c63711dc57e8c1fc Mon Sep 17 00:00:00 2001 From: Florian Schmidt Date: Tue, 13 Mar 2018 16:54:06 +0100 Subject: [PATCH 133/268] [hotfix][docs][py] Fix class name in example This closes #5692. --- docs/dev/stream/python.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/dev/stream/python.md b/docs/dev/stream/python.md index a533819f5026d..887d983c4040e 100644 --- a/docs/dev/stream/python.md +++ b/docs/dev/stream/python.md @@ -464,7 +464,7 @@ Rich functions (.e.g `RichFilterFunction`) enable to define (override) the optio The user may use these functions for initialization and cleanups. {% highlight python %} -class Tockenizer(RichMapFunction): +class Tokenizer(RichMapFunction): def open(self, config): pass def close(self): @@ -472,7 +472,7 @@ class Tockenizer(RichMapFunction): def map(self, value): pass -data_stream.map(Tockenizer()) +data_stream.map(Tokenizer()) {% endhighlight %} The `open` function is called by the worker before starting the streaming pipeline. From 795f59d24a1a8d0decde92a9785efb75883d4cc7 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 9 Mar 2018 10:56:31 +0100 Subject: [PATCH 134/268] [hotfix][cli][tests] let CliFrontendRunTest extend from TestLogger --- .../java/org/apache/flink/client/cli/CliFrontendRunTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java index ebb76d886d711..69724f1008368 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; import org.junit.Test; @@ -37,7 +38,7 @@ /** * Tests for the RUN command. */ -public class CliFrontendRunTest { +public class CliFrontendRunTest extends TestLogger { @BeforeClass public static void init() { From 583bf063ef050216a0432de274e0b82fe554e57c Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 6 Mar 2018 11:43:32 +0100 Subject: [PATCH 135/268] [FLINK-8904][cli][tests] Restore previous sysout This closes #5670. --- .../apache/flink/client/cli/CliFrontendCancelTest.java | 6 ++++++ .../org/apache/flink/client/cli/CliFrontendListTest.java | 6 ++++++ .../flink/client/cli/CliFrontendPackageProgramTest.java | 9 +++++++-- .../org/apache/flink/client/cli/CliFrontendRunTest.java | 6 ++++++ .../org/apache/flink/client/cli/CliFrontendStopTest.java | 9 +++++++-- .../apache/flink/client/cli/CliFrontendTestUtils.java | 6 ++++++ 6 files changed, 38 insertions(+), 4 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java index b2fa003bcb9e1..837c56408699c 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; @@ -46,6 +47,11 @@ public static void init() { CliFrontendTestUtils.pipeSystemOutToNull(); } + @AfterClass + public static void shutdown() { + CliFrontendTestUtils.restoreSystemOut(); + } + @Test public void testCancel() throws Exception { // test cancel properly diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java index 760b376ce7afb..42399cb65f09b 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; @@ -44,6 +45,11 @@ public static void init() { CliFrontendTestUtils.pipeSystemOutToNull(); } + @AfterClass + public static void shutdown() { + CliFrontendTestUtils.restoreSystemOut(); + } + @Test public void testList() throws Exception { // test list properly diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java index 6873e68d1376a..48c889120343c 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java @@ -27,6 +27,7 @@ import org.apache.flink.optimizer.costs.DefaultCostEstimator; import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -40,7 +41,6 @@ import static org.apache.flink.client.cli.CliFrontendTestUtils.TEST_JAR_MAIN_CLASS; import static org.apache.flink.client.cli.CliFrontendTestUtils.getNonJarFilePath; import static org.apache.flink.client.cli.CliFrontendTestUtils.getTestJarPath; -import static org.apache.flink.client.cli.CliFrontendTestUtils.pipeSystemOutToNull; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -58,7 +58,12 @@ public class CliFrontendPackageProgramTest extends TestLogger { @BeforeClass public static void init() { - pipeSystemOutToNull(); + CliFrontendTestUtils.pipeSystemOutToNull(); + } + + @AfterClass + public static void shutdown() { + CliFrontendTestUtils.restoreSystemOut(); } @Before diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java index 69724f1008368..c7789a89ac5c2 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -45,6 +46,11 @@ public static void init() { CliFrontendTestUtils.pipeSystemOutToNull(); } + @AfterClass + public static void shutdown() { + CliFrontendTestUtils.restoreSystemOut(); + } + @Test public void testRun() throws Exception { final Configuration configuration = GlobalConfiguration.loadConfiguration(CliFrontendTestUtils.getConfigDir()); diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java index d6049e55d5ce3..ec4ccdca7d1df 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java @@ -26,6 +26,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; @@ -34,7 +35,6 @@ import java.util.Collections; -import static org.apache.flink.client.cli.CliFrontendTestUtils.pipeSystemOutToNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -49,7 +49,12 @@ public class CliFrontendStopTest extends TestLogger { @BeforeClass public static void setup() { - pipeSystemOutToNull(); + CliFrontendTestUtils.pipeSystemOutToNull(); + } + + @AfterClass + public static void shutdown() { + CliFrontendTestUtils.restoreSystemOut(); } @Test diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestUtils.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestUtils.java index 16737dd125ffd..b47986ff21f8d 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestUtils.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestUtils.java @@ -41,6 +41,8 @@ public class CliFrontendTestUtils { public static final int TEST_JOB_MANAGER_PORT = 55443; + private static final PrintStream previousSysout = System.out; + public static String getTestJarPath() throws FileNotFoundException, MalformedURLException { File f = new File("target/maven-test-jar.jar"); if (!f.exists()) { @@ -68,6 +70,10 @@ public static void pipeSystemOutToNull() { System.setOut(new PrintStream(new BlackholeOutputSteam())); } + public static void restoreSystemOut() { + System.setOut(previousSysout); + } + private static final class BlackholeOutputSteam extends java.io.OutputStream { @Override public void write(int b){} From 882fc65213bdd789e733d13086e3914676666a02 Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Fri, 9 Mar 2018 23:35:15 -0800 Subject: [PATCH 136/268] [hotfix][javadocs] Update javadoc of InternalTimerService.registerEventTimeTimer() This closes #5677. --- .../flink/streaming/api/operators/InternalTimerService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerService.java index f55cb0388b606..cb171fb752ce2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerService.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerService.java @@ -49,7 +49,7 @@ public interface InternalTimerService { void deleteProcessingTimeTimer(N namespace, long time); /** - * Registers a timer to be fired when processing time passes the given time. The namespace + * Registers a timer to be fired when event time watermark passes the given time. The namespace * you pass here will be provided when the timer fires. */ void registerEventTimeTimer(N namespace, long time); From ba43d6bc586bb47c30c2b70f33cadf038f0323ef Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Feb 2018 11:11:59 +0100 Subject: [PATCH 137/268] [FLINK-8703][tests] Port KafkaShortRetentionTestBase to MiniClusterResource This closes #5666. --- .../kafka/KafkaShortRetentionTestBase.java | 41 ++++++++----------- 1 file changed, 17 insertions(+), 24 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index de72985f6b986..15d972f5570f2 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -24,15 +24,14 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper; +import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.util.InstantiationUtil; import org.junit.AfterClass; @@ -68,21 +67,32 @@ public class KafkaShortRetentionTestBase implements Serializable { private static KafkaTestEnvironment kafkaServer; private static Properties standardProps; - private static LocalFlinkMiniCluster flink; + + @ClassRule + public static MiniClusterResource flink = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfiguration(), + NUM_TMS, + TM_SLOTS)); @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); protected static Properties secureProps = new Properties(); + private static Configuration getConfiguration() { + Configuration flinkConfig = new Configuration(); + flinkConfig.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 16L); + flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); + return flinkConfig; + } + @BeforeClass - public static void prepare() throws IOException, ClassNotFoundException { + public static void prepare() throws ClassNotFoundException { LOG.info("-------------------------------------------------------------------------"); LOG.info(" Starting KafkaShortRetentionTestBase "); LOG.info("-------------------------------------------------------------------------"); - Configuration flinkConfig = new Configuration(); - // dynamically load the implementation for the test Class clazz = Class.forName("org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); kafkaServer = (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz); @@ -101,26 +111,10 @@ public static void prepare() throws IOException, ClassNotFoundException { kafkaServer.prepare(kafkaServer.createConfig().setKafkaServerProperties(specificProperties)); standardProps = kafkaServer.getStandardProperties(); - - // start also a re-usable Flink mini cluster - flinkConfig.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); - flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS); - flinkConfig.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 16L); - flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s"); - - flink = new LocalFlinkMiniCluster(flinkConfig, false); - flink.start(); - - TestStreamEnvironment.setAsContext(flink, PARALLELISM); } @AfterClass public static void shutDownServices() throws Exception { - TestStreamEnvironment.unsetAsContext(); - - if (flink != null) { - flink.stop(); - } kafkaServer.shutdown(); secureProps.clear(); @@ -238,8 +232,7 @@ public void runFailOnAutoOffsetResetNone() throws Exception { kafkaServer.createTestTopic(topic, parallelism, 1); - final StreamExecutionEnvironment env = - StreamExecutionEnvironment.createRemoteEnvironment("localhost", flink.getLeaderRPCPort()); + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(parallelism); env.setRestartStrategy(RestartStrategies.noRestart()); // fail immediately env.getConfig().disableSysoutLogging(); From 44f7533db0d0b0791a463d68e28b76fb7622fb88 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Feb 2018 15:19:50 +0100 Subject: [PATCH 138/268] [FLINK-8703][tests] Port NotSoMiniClusterIterations to MiniClusterResource This closes #5667. --- .../manual/NotSoMiniClusterIterations.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java b/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java index 9f6bcbbf67c51..abb8673db5684 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/NotSoMiniClusterIterations.java @@ -24,12 +24,11 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.examples.java.graph.ConnectedComponents; import org.apache.flink.examples.java.graph.util.ConnectedComponentsData; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.test.util.MiniClusterResource; import static org.junit.Assert.fail; @@ -46,23 +45,25 @@ public static void main(String[] args) { throw new RuntimeException("This test program needs to run with at least 5GB of heap space."); } - LocalFlinkMiniCluster cluster = null; + MiniClusterResource cluster = null; try { Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, PARALLELISM); config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 8L); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 1000); config.setInteger(TaskManagerOptions.MEMORY_SEGMENT_SIZE, 8 * 1024); config.setInteger("taskmanager.net.server.numThreads", 1); config.setInteger("taskmanager.net.client.numThreads", 1); - cluster = new LocalFlinkMiniCluster(config, false); - cluster.start(); + cluster = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + config, + PARALLELISM, + 1)); + cluster.before(); - runConnectedComponents(cluster.getLeaderRPCPort()); + runConnectedComponents(); } catch (Exception e) { e.printStackTrace(); @@ -70,14 +71,14 @@ public static void main(String[] args) { } finally { if (cluster != null) { - cluster.stop(); + cluster.after(); } } } - private static void runConnectedComponents(int jmPort) throws Exception { + private static void runConnectedComponents() throws Exception { - ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jmPort); + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.getConfig().disableSysoutLogging(); From 2c850d14c9c11e421ae832e0ea62004f9ef27426 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 27 Feb 2018 15:21:50 +0100 Subject: [PATCH 139/268] [FLINK-8703][tests] Port StreamingScalabilityAndLatency to MiniClusterResource This closes #5668. --- .../StreamingScalabilityAndLatency.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java index efcefebe34a01..a5b01bcd4f444 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java @@ -20,14 +20,13 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.test.util.MiniClusterResource; import static org.junit.Assert.fail; @@ -45,22 +44,24 @@ public static void main(String[] args) throws Exception { final int slotsPerTaskManager = 80; final int parallelism = taskManagers * slotsPerTaskManager; - LocalFlinkMiniCluster cluster = null; + MiniClusterResource cluster = null; try { Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, taskManagers); config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 80L); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, slotsPerTaskManager); config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 20000); config.setInteger("taskmanager.net.server.numThreads", 1); config.setInteger("taskmanager.net.client.numThreads", 1); - cluster = new LocalFlinkMiniCluster(config, false); - cluster.start(); + cluster = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + config, + taskManagers, + slotsPerTaskManager)); + cluster.before(); - runPartitioningProgram(cluster.getLeaderRPCPort(), parallelism); + runPartitioningProgram(parallelism); } catch (Exception e) { e.printStackTrace(); @@ -68,13 +69,13 @@ public static void main(String[] args) throws Exception { } finally { if (cluster != null) { - cluster.stop(); + cluster.after(); } } } - private static void runPartitioningProgram(int jobManagerPort, int parallelism) throws Exception { - StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort); + private static void runPartitioningProgram(int parallelism) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(parallelism); env.getConfig().enableObjectReuse(); From 9b1b7f385f11745544eab21f0383ca3f73e63814 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 13 Mar 2018 13:00:47 +0100 Subject: [PATCH 140/268] [FLINK-4569][tests] Respect exceptions thrown in thread in JobRetrievalITCase This closes #5689. --- .../test/example/client/JobRetrievalITCase.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java index d34b6c337a06e..57198c054b1cc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/client/JobRetrievalITCase.java @@ -42,6 +42,7 @@ import org.junit.Test; import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicReference; import scala.collection.Seq; @@ -85,6 +86,7 @@ public void testJobRetrieval() throws Exception { // has been attached in resumingThread lock.acquire(); client.runDetached(jobGraph, JobRetrievalITCase.class.getClassLoader()); + final AtomicReference error = new AtomicReference<>(); final Thread resumingThread = new Thread(new Runnable() { @Override @@ -92,10 +94,10 @@ public void run() { try { assertNotNull(client.retrieveJob(jobID)); } catch (Throwable e) { - fail(e.getMessage()); + error.set(e); } } - }); + }, "Flink-Job-Retriever"); final Seq actorSystemSeq = cluster.jobManagerActorSystems().get(); final ActorSystem actorSystem = actorSystemSeq.last(); @@ -119,6 +121,11 @@ public void run() { lock.release(); resumingThread.join(); + + Throwable exception = error.get(); + if (exception != null) { + throw new AssertionError(exception); + } } @Test @@ -148,6 +155,7 @@ public SemaphoreInvokable(Environment environment) { @Override public void invoke() throws Exception { lock.acquire(); + lock.release(); } } From 5896840b17d6b39345c836ad2bba0481990432c4 Mon Sep 17 00:00:00 2001 From: Kailash HD Date: Thu, 8 Mar 2018 10:32:23 -0800 Subject: [PATCH 141/268] [FLINK-8888] [Kinesis Connectors] Update the AWS SDK for flink kinesis connector --- flink-connectors/flink-connector-kinesis/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index 99629bfec955a..7ae2e32b468be 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -33,7 +33,7 @@ under the License. flink-connector-kinesis_${scala.binary.version} flink-connector-kinesis - 1.11.171 + 1.11.272 1.8.1 0.12.6 From 7763f7f680fec162e27855f792760bcc3820b799 Mon Sep 17 00:00:00 2001 From: Kailash HD Date: Wed, 14 Mar 2018 09:20:12 -0700 Subject: [PATCH 142/268] [FLINK-8945] [kinesis] Allow customization of KinesisProxy --- .../flink/streaming/connectors/kinesis/proxy/KinesisProxy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java index da81a6575aa44..057e18df0cf94 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/proxy/KinesisProxy.java @@ -122,7 +122,7 @@ public class KinesisProxy implements KinesisProxyInterface { * * @param configProps configuration properties containing AWS credential and AWS region info */ - private KinesisProxy(Properties configProps) { + protected KinesisProxy(Properties configProps) { checkNotNull(configProps); this.kinesisClient = AWSUtil.createKinesisClient(configProps); From c74f80869f3407ca8d02e79fbcf8a5b267ea7253 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Mar 2018 11:05:57 +0100 Subject: [PATCH 143/268] Revert "[FLINK-7851] [scheduling] Improve scheduling balance by round robin distribution" This reverts commit d9c669d4781f095806013651c1a579eae0ca2650. --- .../instance/SlotSharingGroupAssignment.java | 46 +++++------ .../SlotSharingGroupAssignmentTest.java | 79 ------------------- 2 files changed, 20 insertions(+), 105 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignmentTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java index 289762c82e198..e61ba587e5f7d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignment.java @@ -97,7 +97,7 @@ public class SlotSharingGroupAssignment { private final Set allSlots = new LinkedHashSet(); /** The slots available per vertex type (JobVertexId), keyed by TaskManager, to make them locatable */ - private final Map>> availableSlotsPerJid = new LinkedHashMap<>(); + private final Map>> availableSlotsPerJid = new LinkedHashMap<>(); // -------------------------------------------------------------------------------------------- @@ -234,7 +234,7 @@ private SimpleSlot addSharedSlotAndAllocateSubSlot( // can place a task into this slot. boolean entryForNewJidExists = false; - for (Map.Entry>> entry : availableSlotsPerJid.entrySet()) { + for (Map.Entry>> entry : availableSlotsPerJid.entrySet()) { // there is already an entry for this groupID if (entry.getKey().equals(groupIdForMap)) { entryForNewJidExists = true; @@ -247,7 +247,7 @@ private SimpleSlot addSharedSlotAndAllocateSubSlot( // make sure an empty entry exists for this group, if no other entry exists if (!entryForNewJidExists) { - availableSlotsPerJid.put(groupIdForMap, new LinkedHashMap<>()); + availableSlotsPerJid.put(groupIdForMap, new LinkedHashMap>()); } return subSlot; @@ -393,7 +393,7 @@ public Tuple2 getSharedSlotForTask( } // get the available slots for the group - LinkedHashMap> slotsForGroup = availableSlotsPerJid.get(groupId); + Map> slotsForGroup = availableSlotsPerJid.get(groupId); if (slotsForGroup == null) { // we have a new group, so all slots are available @@ -624,26 +624,20 @@ private static SharedSlot removeFromMultiMap(Map> m private static SharedSlot pollFromMultiMap(Map> map) { Iterator>> iter = map.entrySet().iterator(); - + while (iter.hasNext()) { - Map.Entry> slotEntry = iter.next(); - - // remove first entry to add it at the back if there are still slots left - iter.remove(); - - List slots = slotEntry.getValue(); - - if (!slots.isEmpty()) { - - SharedSlot result = slots.remove(slots.size() - 1); - - if (!slots.isEmpty()) { - // reinserts the entry; since it is a LinkedHashMap, we will iterate over this entry - // only after having polled from all other entries - map.put(slotEntry.getKey(), slots); - } - - return result; + List slots = iter.next().getValue(); + + if (slots.isEmpty()) { + iter.remove(); + } + else if (slots.size() == 1) { + SharedSlot slot = slots.remove(0); + iter.remove(); + return slot; + } + else { + return slots.remove(slots.size() - 1); } } @@ -651,11 +645,11 @@ private static SharedSlot pollFromMultiMap(Map> map } private static void removeSlotFromAllEntries( - Map>> availableSlots, - SharedSlot slot) { + Map>> availableSlots, SharedSlot slot) + { final ResourceID taskManagerId = slot.getTaskManagerID(); - for (Map.Entry>> entry : availableSlots.entrySet()) { + for (Map.Entry>> entry : availableSlots.entrySet()) { Map> map = entry.getValue(); List list = map.get(taskManagerId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignmentTest.java deleted file mode 100644 index 2407c1df01906..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotSharingGroupAssignmentTest.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.instance; - -import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobmanager.scheduler.Locality; -import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; -import org.apache.flink.runtime.jobmaster.SlotOwner; -import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.util.Collections; - -import static org.junit.Assert.assertNotEquals; -import static org.junit.Assert.assertNotNull; -import static org.mockito.Mockito.mock; - -public class SlotSharingGroupAssignmentTest extends TestLogger { - - /** - * Tests that slots are allocated in a round robin fashion from the set of available resources. - */ - @Test - public void testRoundRobinPolling() throws UnknownHostException { - final SlotSharingGroupAssignment slotSharingGroupAssignment = new SlotSharingGroupAssignment(); - final int numberTaskManagers = 2; - final int numberSlots = 2; - final JobVertexID sourceId = new JobVertexID(); - final JobVertexID sinkId = new JobVertexID(); - - for (int i = 0; i < numberTaskManagers; i++) { - final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(ResourceID.generate(), InetAddress.getLocalHost(), i + 1000); - - for (int j = 0; j < numberSlots; j++) { - final SharedSlot slot = new SharedSlot( - mock(SlotOwner.class), - taskManagerLocation, - j, - mock(TaskManagerGateway.class), - slotSharingGroupAssignment); - - slotSharingGroupAssignment.addSharedSlotAndAllocateSubSlot(slot, Locality.UNKNOWN, sourceId); - } - } - - SimpleSlot allocatedSlot1 = slotSharingGroupAssignment.getSlotForTask(sinkId, Collections.emptyList()); - SimpleSlot allocatedSlot2 = slotSharingGroupAssignment.getSlotForTask(sinkId, Collections.emptyList()); - - assertNotEquals(allocatedSlot1.getTaskManagerLocation(), allocatedSlot2.getTaskManagerLocation()); - - // let's check that we can still allocate all 4 slots - SimpleSlot allocatedSlot3 = slotSharingGroupAssignment.getSlotForTask(sinkId, Collections.emptyList()); - assertNotNull(allocatedSlot3); - - SimpleSlot allocatedSlot4 = slotSharingGroupAssignment.getSlotForTask(sinkId, Collections.emptyList()); - assertNotNull(allocatedSlot4); - } -} From c2dd43e370e753ee5b57082ef4716c36575e6fac Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 16 Mar 2018 19:16:49 +0100 Subject: [PATCH 144/268] [FLINK-9016] [flip6] Properly unregister jobs from JobMetricGroup This commit properly removes jobs from the JobMetricGroup once a job has reached a terminal state. --- .../flink/runtime/dispatcher/Dispatcher.java | 33 +++++++++++++------ .../runtime/dispatcher/MiniDispatcher.java | 8 +++-- .../dispatcher/StandaloneDispatcher.java | 8 +++-- .../runtime/entrypoint/ClusterEntrypoint.java | 27 +++++++++++++-- .../entrypoint/JobClusterEntrypoint.java | 8 +++-- .../entrypoint/SessionClusterEntrypoint.java | 26 ++++++++------- .../runtime/jobmaster/JobManagerRunner.java | 31 +++-------------- .../flink/runtime/jobmaster/JobMaster.java | 15 +++------ .../runtime/minicluster/MiniCluster.java | 14 +++++++- .../runtime/dispatcher/DispatcherTest.java | 20 +++++++---- .../dispatcher/MiniDispatcherTest.java | 10 +++--- .../jobmaster/JobManagerRunnerTest.java | 10 ++---- .../runtime/jobmaster/JobMasterTest.java | 3 +- 13 files changed, 122 insertions(+), 91 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 9b2411c66b789..91a4f73bf5384 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -49,7 +49,8 @@ import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.messages.webmonitor.JobsOverview; import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceOverview; import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse; @@ -97,7 +98,6 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final JobManagerSharedServices jobManagerSharedServices; private final HeartbeatServices heartbeatServices; private final BlobServer blobServer; - private final MetricRegistry metricRegistry; private final FatalErrorHandler fatalErrorHandler; @@ -109,6 +109,11 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final JobManagerRunnerFactory jobManagerRunnerFactory; + private final JobManagerMetricGroup jobManagerMetricGroup; + + @Nullable + private final String metricQueryServicePath; + @Nullable protected final String restAddress; @@ -123,7 +128,8 @@ public Dispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricServiceQueryPath, ArchivedExecutionGraphStore archivedExecutionGraphStore, JobManagerRunnerFactory jobManagerRunnerFactory, FatalErrorHandler fatalErrorHandler, @@ -135,9 +141,10 @@ public Dispatcher( this.resourceManagerGateway = Preconditions.checkNotNull(resourceManagerGateway); this.heartbeatServices = Preconditions.checkNotNull(heartbeatServices); this.blobServer = Preconditions.checkNotNull(blobServer); - this.metricRegistry = Preconditions.checkNotNull(metricRegistry); this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler); this.submittedJobGraphStore = Preconditions.checkNotNull(submittedJobGraphStore); + this.jobManagerMetricGroup = Preconditions.checkNotNull(jobManagerMetricGroup); + this.metricQueryServicePath = metricServiceQueryPath; this.jobManagerSharedServices = JobManagerSharedServices.fromConfiguration( configuration, @@ -192,6 +199,8 @@ public CompletableFuture postStop() { exception = ExceptionUtils.firstOrSuppressed(e, exception); } + jobManagerMetricGroup.close(); + if (exception != null) { throw exception; } else { @@ -251,7 +260,8 @@ public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) heartbeatServices, blobServer, jobManagerSharedServices, - metricRegistry, + jobManagerMetricGroup.addJob(jobGraph), + metricQueryServicePath, restAddress); jobManagerRunner.getResultFuture().whenCompleteAsync( @@ -464,8 +474,6 @@ public CompletableFuture requestJobResult(JobID jobId, Time timeout) @Override public CompletableFuture> requestMetricQueryServicePaths(Time timeout) { - final String metricQueryServicePath = metricRegistry.getMetricQueryServicePath(); - if (metricQueryServicePath != null) { return CompletableFuture.completedFuture(Collections.singleton(metricQueryServicePath)); } else { @@ -513,6 +521,8 @@ private void removeJob(JobID jobId, boolean cleanupHA) throws Exception { registerOrphanedJobManagerTerminationFuture(jobManagerRunnerTerminationFuture); } + jobManagerMetricGroup.removeJob(jobId); + if (cleanupHA) { submittedJobGraphStore.removeJobGraph(jobId); } @@ -725,7 +735,8 @@ JobManagerRunner createJobManagerRunner( HeartbeatServices heartbeatServices, BlobServer blobServer, JobManagerSharedServices jobManagerServices, - MetricRegistry metricRegistry, + JobManagerJobMetricGroup jobManagerJobMetricGroup, + @Nullable String metricQueryServicePath, @Nullable String restAddress) throws Exception; } @@ -745,7 +756,8 @@ public JobManagerRunner createJobManagerRunner( HeartbeatServices heartbeatServices, BlobServer blobServer, JobManagerSharedServices jobManagerServices, - MetricRegistry metricRegistry, + JobManagerJobMetricGroup jobManagerJobMetricGroup, + @Nullable String metricQueryServicePath, @Nullable String restAddress) throws Exception { return new JobManagerRunner( resourceId, @@ -756,7 +768,8 @@ public JobManagerRunner createJobManagerRunner( heartbeatServices, blobServer, jobManagerServices, - metricRegistry, + jobManagerJobMetricGroup, + metricQueryServicePath, restAddress); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java index c648131a7c09e..3f458248fff77 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -30,7 +30,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -60,7 +60,8 @@ public MiniDispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricQueryServicePath, ArchivedExecutionGraphStore archivedExecutionGraphStore, JobManagerRunnerFactory jobManagerRunnerFactory, FatalErrorHandler fatalErrorHandler, @@ -76,7 +77,8 @@ public MiniDispatcher( resourceManagerGateway, blobServer, heartbeatServices, - metricRegistry, + jobManagerMetricGroup, + metricQueryServicePath, archivedExecutionGraphStore, jobManagerRunnerFactory, fatalErrorHandler, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java index a7d21f3faaa08..52ac7a0606a57 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobMaster; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -45,7 +45,8 @@ public StandaloneDispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricQueryServicePath, ArchivedExecutionGraphStore archivedExecutionGraphStore, JobManagerRunnerFactory jobManagerRunnerFactory, FatalErrorHandler fatalErrorHandler, @@ -59,7 +60,8 @@ public StandaloneDispatcher( resourceManagerGateway, blobServer, heartbeatServices, - metricRegistry, + jobManagerMetricGroup, + metricQueryServicePath, archivedExecutionGraphStore, jobManagerRunnerFactory, fatalErrorHandler, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 07b3b683a3f28..676415b18e1ff 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -51,6 +51,8 @@ import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.MetricRegistryImpl; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.metrics.util.MetricUtils; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; @@ -154,6 +156,9 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler { @GuardedBy("lock") private ClusterInformation clusterInformation; + @GuardedBy("lock") + private JobManagerMetricGroup jobManagerMetricGroup; + protected ClusterEntrypoint(Configuration configuration) { this.configuration = Preconditions.checkNotNull(configuration); this.terminationFuture = new CompletableFuture<>(); @@ -327,6 +332,8 @@ protected void startClusterComponents( clusterInformation, webMonitorEndpoint.getRestAddress()); + jobManagerMetricGroup = MetricUtils.instantiateJobManagerMetricGroup(metricRegistry, rpcService.getAddress()); + dispatcher = createDispatcher( configuration, rpcService, @@ -334,7 +341,8 @@ protected void startClusterComponents( resourceManager.getSelfGateway(ResourceManagerGateway.class), blobServer, heartbeatServices, - metricRegistry, + jobManagerMetricGroup, + metricRegistry.getMetricQueryServicePath(), archivedExecutionGraphStore, this, webMonitorEndpoint.getRestAddress()); @@ -488,7 +496,19 @@ protected CompletableFuture stopClusterComponents() { terminationFutures.add(FutureUtils.completedExceptionally(exception)); } - return FutureUtils.completeAll(terminationFutures); + final CompletableFuture componentTerminationFuture = FutureUtils.completeAll(terminationFutures); + + if (jobManagerMetricGroup != null) { + return FutureUtils.runAfterwards( + componentTerminationFuture, + () -> { + synchronized (lock) { + jobManagerMetricGroup.close(); + } + }); + } else { + return componentTerminationFuture; + } } } @@ -567,7 +587,8 @@ protected abstract Dispatcher createDispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricQueryServicePath, ArchivedExecutionGraphStore archivedExecutionGraphStore, FatalErrorHandler fatalErrorHandler, @Nullable String restAddress) throws Exception; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java index dc211d8406053..df950a343be94 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; @@ -95,7 +95,8 @@ protected Dispatcher createDispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricQueryServicePath, ArchivedExecutionGraphStore archivedExecutionGraphStore, FatalErrorHandler fatalErrorHandler, @Nullable String restAddress) throws Exception { @@ -114,7 +115,8 @@ protected Dispatcher createDispatcher( resourceManagerGateway, blobServer, heartbeatServices, - metricRegistry, + jobManagerMetricGroup, + metricQueryServicePath, archivedExecutionGraphStore, Dispatcher.DefaultJobManagerRunnerFactory.INSTANCE, fatalErrorHandler, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java index 764356d036a85..fcab796ffd54e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/SessionClusterEntrypoint.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.leaderelection.LeaderElectionService; -import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rest.RestServerEndpointConfiguration; import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration; @@ -104,16 +104,17 @@ protected DispatcherRestEndpoint createRestEndpoint( @Override protected Dispatcher createDispatcher( - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - ResourceManagerGateway resourceManagerGateway, - BlobServer blobServer, - HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, - ArchivedExecutionGraphStore archivedExecutionGraphStore, - FatalErrorHandler fatalErrorHandler, - @Nullable String restAddress) throws Exception { + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + ResourceManagerGateway resourceManagerGateway, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricQueryServicePath, + ArchivedExecutionGraphStore archivedExecutionGraphStore, + FatalErrorHandler fatalErrorHandler, + @Nullable String restAddress) throws Exception { // create the default dispatcher return new StandaloneDispatcher( @@ -124,7 +125,8 @@ protected Dispatcher createDispatcher( resourceManagerGateway, blobServer, heartbeatServices, - metricRegistry, + jobManagerMetricGroup, + metricQueryServicePath, archivedExecutionGraphStore, Dispatcher.DefaultJobManagerRunnerFactory.INSTANCE, fatalErrorHandler, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java index 8b64f0ddf57e9..cd2852b7fae18 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerRunner.java @@ -35,9 +35,7 @@ import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.metrics.util.MetricUtils; +import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.AutoCloseableAsync; @@ -82,8 +80,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F private final JobMaster jobManager; - private final JobManagerMetricGroup jobManagerMetricGroup; - private final Time rpcTimeout; private final CompletableFuture resultFuture; @@ -111,11 +107,10 @@ public JobManagerRunner( final HeartbeatServices heartbeatServices, final BlobServer blobServer, final JobManagerSharedServices jobManagerSharedServices, - final MetricRegistry metricRegistry, + final JobManagerJobMetricGroup jobManagerJobMetricGroup, + @Nullable final String metricQueryServicePath, @Nullable final String restAddress) throws Exception { - JobManagerMetricGroup jobManagerMetrics = null; - this.resultFuture = new CompletableFuture<>(); this.terminationFuture = new CompletableFuture<>(); @@ -126,10 +121,6 @@ public JobManagerRunner( checkArgument(jobGraph.getNumberOfVertices() > 0, "The given job is empty"); - final String hostAddress = rpcService.getAddress().isEmpty() ? "localhost" : rpcService.getAddress(); - jobManagerMetrics = MetricUtils.instantiateJobManagerMetricGroup(metricRegistry, hostAddress); - this.jobManagerMetricGroup = jobManagerMetrics; - // libraries and class loader first final LibraryCacheManager libraryCacheManager = jobManagerSharedServices.getLibraryCacheManager(); try { @@ -162,19 +153,14 @@ public JobManagerRunner( jobManagerSharedServices, heartbeatServices, blobServer, - jobManagerMetrics, + jobManagerJobMetricGroup, this, this, userCodeLoader, restAddress, - metricRegistry.getMetricQueryServicePath()); + metricQueryServicePath); } catch (Throwable t) { - // clean up everything - if (jobManagerMetrics != null) { - jobManagerMetrics.close(); - } - terminationFuture.completeExceptionally(t); resultFuture.completeExceptionally(t); @@ -230,13 +216,6 @@ public CompletableFuture closeAsync() { throwable = ExceptionUtils.firstOrSuppressed(t, ExceptionUtils.stripCompletionException(throwable)); } - // make all registered metrics go away - try { - jobManagerMetricGroup.close(); - } catch (Throwable t) { - throwable = ExceptionUtils.firstOrSuppressed(t, throwable); - } - if (throwable != null) { terminationFuture.completeExceptionally( new FlinkException("Could not properly shut down the JobManagerRunner", throwable)); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 74f9b656c6f0e..ced8c7c4dd89e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -24,7 +24,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; -import org.apache.flink.metrics.MetricGroup; import org.apache.flink.queryablestate.KvStateID; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.StoppingException; @@ -76,8 +75,7 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.messages.webmonitor.JobDetails; -import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.query.KvStateLocationRegistry; import org.apache.flink.runtime.query.UnknownKvStateLocation; @@ -165,7 +163,7 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast private final BlobServer blobServer; /** The metrics for the job. */ - private final MetricGroup jobMetricGroup; + private final JobManagerJobMetricGroup jobMetricGroup; /** The heartbeat manager with task managers. */ private final HeartbeatManager taskManagerHeartbeatManager; @@ -225,7 +223,7 @@ public JobMaster( JobManagerSharedServices jobManagerSharedServices, HeartbeatServices heartbeatServices, BlobServer blobServer, - @Nullable JobManagerMetricGroup jobManagerMetricGroup, + JobManagerJobMetricGroup jobMetricGroup, OnCompletionActions jobCompletionActions, FatalErrorHandler errorHandler, ClassLoader userCodeLoader, @@ -246,6 +244,7 @@ public JobMaster( this.jobCompletionActions = checkNotNull(jobCompletionActions); this.errorHandler = checkNotNull(errorHandler); this.userCodeLoader = checkNotNull(userCodeLoader); + this.jobMetricGroup = checkNotNull(jobMetricGroup); this.taskManagerHeartbeatManager = heartbeatServices.createHeartbeatManagerSender( resourceId, @@ -262,12 +261,6 @@ public JobMaster( final String jobName = jobGraph.getName(); final JobID jid = jobGraph.getJobID(); - if (jobManagerMetricGroup != null) { - this.jobMetricGroup = jobManagerMetricGroup.addJob(jobGraph); - } else { - this.jobMetricGroup = UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup(); - } - log.info("Initializing job {} ({}).", jobName, jid); final RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration = diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 98c8ca22e9ba4..d660c6758b3d3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -50,6 +50,8 @@ import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.MetricRegistryImpl; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.metrics.util.MetricUtils; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.ResourceManagerRunner; @@ -153,6 +155,9 @@ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { @GuardedBy("lock") private StandaloneDispatcher dispatcher; + @GuardedBy("lock") + private JobManagerMetricGroup jobManagerMetricGroup; + @GuardedBy("lock") private RpcGatewayRetriever dispatcherGatewayRetriever; @@ -344,6 +349,8 @@ public void start() throws Exception { // bring up the dispatcher that launches JobManagers when jobs submitted LOG.info("Starting job dispatcher(s) for JobManger"); + this.jobManagerMetricGroup = MetricUtils.instantiateJobManagerMetricGroup(metricRegistry, "localhost"); + dispatcher = new StandaloneDispatcher( jobManagerRpcService, Dispatcher.DISPATCHER_NAME + UUID.randomUUID(), @@ -352,7 +359,8 @@ public void start() throws Exception { resourceManagerRunner.getResourceManageGateway(), blobServer, heartbeatServices, - metricRegistry, + jobManagerMetricGroup, + metricRegistry.getMetricQueryServicePath(), new MemoryArchivedExecutionGraphStore(), Dispatcher.DefaultJobManagerRunnerFactory.INSTANCE, new ShutDownFatalErrorHandler(), @@ -424,6 +432,10 @@ public CompletableFuture closeAsync() { componentsTerminationFuture, () -> { synchronized (lock) { + if (jobManagerMetricGroup != null) { + jobManagerMetricGroup.close(); + jobManagerMetricGroup = null; + } // metrics shutdown if (metricRegistry != null) { metricRegistry.shutdown(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 82679216205fc..71c391f20a7c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -45,8 +45,9 @@ import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; -import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.NoOpMetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -177,7 +178,8 @@ public void setUp() throws Exception { mock(ResourceManagerGateway.class), new BlobServer(blobServerConfig, new VoidBlobStore()), heartbeatServices, - NoOpMetricRegistry.INSTANCE, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + null, new MemoryArchivedExecutionGraphStore(), fatalErrorHandler, TEST_JOB_ID); @@ -360,7 +362,8 @@ private TestingDispatcher( ResourceManagerGateway resourceManagerGateway, BlobServer blobServer, HeartbeatServices heartbeatServices, - MetricRegistry metricRegistry, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricQueryServicePath, ArchivedExecutionGraphStore archivedExecutionGraphStore, FatalErrorHandler fatalErrorHandler, JobID expectedJobId) throws Exception { @@ -373,7 +376,8 @@ private TestingDispatcher( resourceManagerGateway, blobServer, heartbeatServices, - metricRegistry, + jobManagerMetricGroup, + metricQueryServicePath, archivedExecutionGraphStore, new ExpectedJobIdJobManagerRunnerFactory(expectedJobId), fatalErrorHandler, @@ -421,7 +425,8 @@ public JobManagerRunner createJobManagerRunner( HeartbeatServices heartbeatServices, BlobServer blobServer, JobManagerSharedServices jobManagerSharedServices, - MetricRegistry metricRegistry, + JobManagerJobMetricGroup jobManagerJobMetricGroup, + @Nullable String metricQueryServicePath, @Nullable String restAddress) throws Exception { assertEquals(expectedJobId, jobGraph.getJobID()); @@ -434,7 +439,8 @@ public JobManagerRunner createJobManagerRunner( heartbeatServices, blobServer, jobManagerSharedServices, - metricRegistry, + jobManagerJobMetricGroup, + metricQueryServicePath, restAddress); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index c6eda2e816a41..651200f6096e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -35,8 +35,8 @@ import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.NoOpMetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; import org.apache.flink.runtime.rpc.RpcService; @@ -254,7 +254,8 @@ private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode exec resourceManagerGateway, blobServer, heartbeatServices, - NoOpMetricRegistry.INSTANCE, + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), + null, archivedExecutionGraphStore, testingJobManagerRunnerFactory, testingFatalErrorHandler, @@ -283,7 +284,8 @@ public JobManagerRunner createJobManagerRunner( HeartbeatServices heartbeatServices, BlobServer blobServer, JobManagerSharedServices jobManagerSharedServices, - MetricRegistry metricRegistry, + JobManagerJobMetricGroup jobManagerJobMetricGroup, + @Nullable String metricQueryServicePath, @Nullable String restAddress) throws Exception { jobGraphFuture.complete(jobGraph); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java index 9730ddef27e5a..1d7f0906f4308 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerTest.java @@ -32,8 +32,7 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; -import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.metrics.NoOpMetricRegistry; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testtasks.NoOpInvokable; @@ -76,8 +75,6 @@ public class JobManagerRunnerTest extends TestLogger { private static JobManagerSharedServices jobManagerSharedServices; - private static MetricRegistry metricRegistry; - private static JobGraph jobGraph; private static ArchivedExecutionGraph archivedExecutionGraph; @@ -97,8 +94,6 @@ public static void setupClass() throws Exception { jobManagerSharedServices = JobManagerSharedServices.fromConfiguration(configuration, blobServer); - metricRegistry = NoOpMetricRegistry.INSTANCE; - final JobVertex jobVertex = new JobVertex("Test vertex"); jobVertex.setInvokableClass(NoOpInvokable.class); jobGraph = new JobGraph(jobVertex); @@ -215,7 +210,8 @@ private JobManagerRunner createJobManagerRunner() throws Exception { heartbeatServices, blobServer, jobManagerSharedServices, - metricRegistry, + UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup(), + null, null); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index b5430569e1913..ed5a8946b1caa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -47,6 +47,7 @@ import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; @@ -410,7 +411,7 @@ private JobMaster createJobMaster( jobManagerSharedServices, fastHeartbeatServices, blobServer, - null, + UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup(), new NoOpOnCompletionActions(), testingFatalErrorHandler, JobMasterTest.class.getClassLoader(), From d3b83a9da854b73b42ce68f78f46a8a11c510914 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 17 Mar 2018 17:53:35 +0100 Subject: [PATCH 145/268] [FLINK-8812] [flip6] Set managed memory for TaskExecutor to 80 MB in MiniCluster In order to avoid problems with OOM exceptions, this commit sets the managed memory to 80 MB for TaskExecutors started by the MiniCluster. This closes #5713. --- .../java/org/apache/flink/test/util/MiniClusterResource.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java index 2f12bdc347e84..dbd292cd2d07b 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.minicluster.JobExecutorService; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; @@ -194,6 +195,10 @@ private void startFlip6MiniCluster() throws Exception { configuration.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); } + if (!configuration.contains(TaskManagerOptions.MANAGED_MEMORY_SIZE)) { + configuration.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, TestBaseUtils.TASK_MANAGER_MEMORY_SIZE); + } + // set rest port to 0 to avoid clashes with concurrent MiniClusters configuration.setInteger(RestOptions.REST_PORT, 0); From eb666d8128ca5f63d735d87235959719c141fe93 Mon Sep 17 00:00:00 2001 From: gyao Date: Fri, 9 Mar 2018 14:36:33 +0100 Subject: [PATCH 146/268] [FLINK-7804][flip6] Run AMRMClientAsync callbacks in main thread This closes #5675. --- .../flink/yarn/YarnResourceManager.java | 107 ++++++++++-------- .../flink/yarn/YarnResourceManagerTest.java | 15 +-- 2 files changed, 65 insertions(+), 57 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index af789baf7e95a..97db2ad8a37a1 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -316,7 +316,10 @@ protected YarnWorkerNode workerStarted(ResourceID resourceID) { return workerNodeMap.get(resourceID); } - // AMRMClientAsync CallbackHandler methods + // ------------------------------------------------------------------------ + // AMRMClientAsync CallbackHandler methods + // ------------------------------------------------------------------------ + @Override public float getProgress() { // Temporarily need not record the total size of asked and allocated containers @@ -325,67 +328,68 @@ public float getProgress() { @Override public void onContainersCompleted(List list) { - for (ContainerStatus container : list) { - if (container.getExitStatus() < 0) { - closeTaskManagerConnection(new ResourceID( - container.getContainerId().toString()), new Exception(container.getDiagnostics())); + runAsync(() -> { + for (ContainerStatus container : list) { + if (container.getExitStatus() < 0) { + closeTaskManagerConnection(new ResourceID( + container.getContainerId().toString()), new Exception(container.getDiagnostics())); + } + workerNodeMap.remove(new ResourceID(container.getContainerId().toString())); + } } - workerNodeMap.remove(new ResourceID(container.getContainerId().toString())); - } + ); } @Override public void onContainersAllocated(List containers) { - for (Container container : containers) { - log.info( - "Received new container: {} - Remaining pending container requests: {}", - container.getId(), - numPendingContainerRequests); - - if (numPendingContainerRequests > 0) { - numPendingContainerRequests--; - - final String containerIdStr = container.getId().toString(); - - workerNodeMap.put(new ResourceID(containerIdStr), new YarnWorkerNode(container)); - - try { - // Context information used to start a TaskExecutor Java process - ContainerLaunchContext taskExecutorLaunchContext = createTaskExecutorLaunchContext( - container.getResource(), - containerIdStr, - container.getNodeId().getHost()); - - nodeManagerClient.startContainer(container, taskExecutorLaunchContext); - } catch (Throwable t) { - log.error("Could not start TaskManager in container {}.", container.getId(), t); - - // release the failed container + runAsync(() -> { + for (Container container : containers) { + log.info( + "Received new container: {} - Remaining pending container requests: {}", + container.getId(), + numPendingContainerRequests); + + if (numPendingContainerRequests > 0) { + numPendingContainerRequests--; + + final String containerIdStr = container.getId().toString(); + + workerNodeMap.put(new ResourceID(containerIdStr), new YarnWorkerNode(container)); + + try { + // Context information used to start a TaskExecutor Java process + ContainerLaunchContext taskExecutorLaunchContext = createTaskExecutorLaunchContext( + container.getResource(), + containerIdStr, + container.getNodeId().getHost()); + + nodeManagerClient.startContainer(container, taskExecutorLaunchContext); + } catch (Throwable t) { + log.error("Could not start TaskManager in container {}.", container.getId(), t); + + // release the failed container + resourceManagerClient.releaseAssignedContainer(container.getId()); + // and ask for a new one + requestYarnContainer(container.getResource(), container.getPriority()); + } + } else { + // return the excessive containers + log.info("Returning excess container {}.", container.getId()); resourceManagerClient.releaseAssignedContainer(container.getId()); - // and ask for a new one - requestYarnContainer(container.getResource(), container.getPriority()); } - } else { - // return the excessive containers - log.info("Returning excess container {}.", container.getId()); - resourceManagerClient.releaseAssignedContainer(container.getId()); } - } - // if we are waiting for no further containers, we can go to the - // regular heartbeat interval - if (numPendingContainerRequests <= 0) { - resourceManagerClient.setHeartbeatInterval(yarnHeartbeatIntervalMillis); - } + // if we are waiting for no further containers, we can go to the + // regular heartbeat interval + if (numPendingContainerRequests <= 0) { + resourceManagerClient.setHeartbeatInterval(yarnHeartbeatIntervalMillis); + } + }); } @Override public void onShutdownRequest() { - try { - shutDown(); - } catch (Exception e) { - log.warn("Fail to shutdown the YARN resource manager.", e); - } + shutDown(); } @Override @@ -398,7 +402,10 @@ public void onError(Throwable error) { onFatalError(error); } - //Utility methods + // ------------------------------------------------------------------------ + // Utility methods + // ------------------------------------------------------------------------ + /** * Converts a Flink application status enum to a YARN application status enum. * @param status The Flink application status. diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java index 455abc9596e9c..0d37b8ed6bf98 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnResourceManagerTest.java @@ -72,8 +72,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import javax.annotation.Nullable; @@ -105,14 +103,12 @@ */ public class YarnResourceManagerTest extends TestLogger { - private static final Logger LOG = LoggerFactory.getLogger(YarnResourceManagerTest.class); + private static final Time TIMEOUT = Time.seconds(10L); private static Configuration flinkConfig = new Configuration(); private static Map env = new HashMap<>(); - private static final Time timeout = Time.seconds(10L); - @Rule public TemporaryFolder folder = new TemporaryFolder(); @@ -178,7 +174,7 @@ public TestingYarnResourceManager( } public CompletableFuture runInMainThread(Callable callable) { - return callAsync(callable, timeout); + return callAsync(callable, TIMEOUT); } public MainThreadExecutor getMainThreadExecutorForTesting() { @@ -197,6 +193,11 @@ protected AMRMClientAsync createAndStartResourceMan protected NMClient createAndStartNodeManagerClient(YarnConfiguration yarnConfiguration) { return mockNMClient; } + + @Override + protected void runAsync(final Runnable runnable) { + runnable.run(); + } } static class Context { @@ -292,7 +293,7 @@ class MockResourceManagerRuntimeServices { public void grantLeadership() throws Exception { rmLeaderSessionId = UUID.randomUUID(); - rmLeaderElectionService.isLeader(rmLeaderSessionId).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + rmLeaderElectionService.isLeader(rmLeaderSessionId).get(TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); } } From cb1224bf77cea86d87a990c6b0c9c79a3eda6ac8 Mon Sep 17 00:00:00 2001 From: gyao Date: Fri, 16 Mar 2018 06:57:24 +0100 Subject: [PATCH 147/268] [hotfix][flip6] Only create new terminationFuture if MiniCluster is running --- .../org/apache/flink/runtime/minicluster/MiniCluster.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index d660c6758b3d3..74aa388628119 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -217,9 +217,6 @@ public void start() throws Exception { LOG.info("Starting Flink Mini Cluster"); LOG.debug("Using configuration {}", miniClusterConfiguration); - // create a new termination future - terminationFuture = new CompletableFuture<>(); - final Configuration configuration = miniClusterConfiguration.getConfiguration(); final Time rpcTimeout = miniClusterConfiguration.getRpcTimeout(); final int numTaskManagers = miniClusterConfiguration.getNumTaskManagers(); @@ -384,6 +381,9 @@ public void start() throws Exception { throw e; } + // create a new termination future + terminationFuture = new CompletableFuture<>(); + // now officially mark this as running running = true; From 9c3378a83f0bc18b666c0a4e25dea5a7b9678f99 Mon Sep 17 00:00:00 2001 From: gyao Date: Thu, 15 Mar 2018 22:04:58 +0100 Subject: [PATCH 148/268] [FLINK-8843][REST] Decouple bind REST address from advertised address By default bind REST server on wildcard address. Rename RestServerEndpoint#getRestAddress to getRestBaseUrl. This closes #5707. --- .../flink/configuration/RestOptions.java | 15 ++++-- .../runtime/entrypoint/ClusterEntrypoint.java | 4 +- .../HighAvailabilityServicesUtils.java | 6 ++- .../runtime/minicluster/MiniCluster.java | 4 +- .../minicluster/MiniClusterConfiguration.java | 4 ++ .../runtime/rest/RestServerEndpoint.java | 46 +++++++++++-------- .../rest/RestServerEndpointConfiguration.java | 32 +++++++++---- .../webmonitor/WebMonitorEndpoint.java | 6 +-- .../rest/RestServerEndpointITCase.java | 3 +- 9 files changed, 80 insertions(+), 40 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java index 94d7977b72541..e7421c4dcffdd 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/RestOptions.java @@ -29,12 +29,21 @@ public class RestOptions { /** - * The address that the server binds itself to / the client connects to. + * The address that the server binds itself to. + */ + public static final ConfigOption REST_BIND_ADDRESS = + key("rest.bind-address") + .noDefaultValue() + .withDescription("The address that the server binds itself."); + + /** + * The address that should be used by clients to connect to the server. */ public static final ConfigOption REST_ADDRESS = key("rest.address") - .defaultValue("localhost") - .withDescription("The address that the server binds itself to / the client connects to."); + .noDefaultValue() + .withDeprecatedKeys(JobManagerOptions.ADDRESS.key()) + .withDescription("The address that should be used by clients to connect to the server."); /** * The port that the server listens on / the client connects to. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 676415b18e1ff..63c8072a82620 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -330,7 +330,7 @@ protected void startClusterComponents( metricRegistry, this, clusterInformation, - webMonitorEndpoint.getRestAddress()); + webMonitorEndpoint.getRestBaseUrl()); jobManagerMetricGroup = MetricUtils.instantiateJobManagerMetricGroup(metricRegistry, rpcService.getAddress()); @@ -345,7 +345,7 @@ protected void startClusterComponents( metricRegistry.getMetricQueryServicePath(), archivedExecutionGraphStore, this, - webMonitorEndpoint.getRestAddress()); + webMonitorEndpoint.getRestBaseUrl()); LOG.debug("Starting ResourceManager."); resourceManager.start(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java index 4f12f2bc4816d..f19a421a0c708 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServicesUtils.java @@ -39,6 +39,8 @@ import java.util.concurrent.Executor; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Utils class to instantiate {@link HighAvailabilityServices} implementations. */ @@ -97,7 +99,9 @@ public static HighAvailabilityServices createHighAvailabilityServices( addressResolution, configuration); - final String address = configuration.getString(RestOptions.REST_ADDRESS); + final String address = checkNotNull(configuration.getString(RestOptions.REST_ADDRESS), + "%s must be set", + RestOptions.REST_ADDRESS.key()); final int port = configuration.getInteger(RestOptions.REST_PORT); final boolean enableSSL = configuration.getBoolean(SecurityOptions.SSL_ENABLED); final String protocol = enableSSL ? "https://" : "http://"; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 74aa388628119..dfe30afda5eb6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -341,7 +341,7 @@ public void start() throws Exception { dispatcherRestEndpoint.start(); - restAddressURI = new URI(dispatcherRestEndpoint.getRestAddress()); + restAddressURI = new URI(dispatcherRestEndpoint.getRestBaseUrl()); // bring up the dispatcher that launches JobManagers when jobs submitted LOG.info("Starting job dispatcher(s) for JobManger"); @@ -361,7 +361,7 @@ public void start() throws Exception { new MemoryArchivedExecutionGraphStore(), Dispatcher.DefaultJobManagerRunnerFactory.INSTANCE, new ShutDownFatalErrorHandler(), - dispatcherRestEndpoint.getRestAddress()); + dispatcherRestEndpoint.getRestBaseUrl()); dispatcher.start(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java index 08af0c4fe89f6..fe7669443365c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterConfiguration.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.UnmodifiableConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.util.Preconditions; @@ -167,6 +168,9 @@ public Builder setCommonBindAddress(String commonBindAddress) { public MiniClusterConfiguration build() { final Configuration modifiedConfiguration = new Configuration(configuration); modifiedConfiguration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTaskManager); + modifiedConfiguration.setString( + RestOptions.REST_ADDRESS, + modifiedConfiguration.getString(RestOptions.REST_ADDRESS, "localhost")); return new MiniClusterConfiguration( modifiedConfiguration, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index a3d48431f81e4..dfb01ca2657c9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -73,8 +73,9 @@ public abstract class RestServerEndpoint { private final Object lock = new Object(); - private final String configuredAddress; - private final int configuredPort; + private final String restAddress; + private final String restBindAddress; + private final int restBindPort; private final SSLEngine sslEngine; private final Path uploadDir; private final int maxContentLength; @@ -84,14 +85,16 @@ public abstract class RestServerEndpoint { private ServerBootstrap bootstrap; private Channel serverChannel; - private String restAddress; + private String restBaseUrl; private State state = State.CREATED; public RestServerEndpoint(RestServerEndpointConfiguration configuration) throws IOException { Preconditions.checkNotNull(configuration); - this.configuredAddress = configuration.getEndpointBindAddress(); - this.configuredPort = configuration.getEndpointBindPort(); + + this.restAddress = configuration.getRestAddress(); + this.restBindAddress = configuration.getRestBindAddress(); + this.restBindPort = configuration.getRestBindPort(); this.sslEngine = configuration.getSslEngine(); this.uploadDir = configuration.getUploadDir(); @@ -101,8 +104,6 @@ public RestServerEndpoint(RestServerEndpointConfiguration configuration) throws this.responseHeaders = configuration.getResponseHeaders(); terminationFuture = new CompletableFuture<>(); - - this.restAddress = null; } /** @@ -176,18 +177,23 @@ protected void initChannel(SocketChannel ch) { .childHandler(initializer); final ChannelFuture channel; - if (configuredAddress == null) { - channel = bootstrap.bind(configuredPort); + if (restBindAddress == null) { + channel = bootstrap.bind(restBindPort); } else { - channel = bootstrap.bind(configuredAddress, configuredPort); + channel = bootstrap.bind(restBindAddress, restBindPort); } serverChannel = channel.syncUninterruptibly().channel(); - InetSocketAddress bindAddress = (InetSocketAddress) serverChannel.localAddress(); - String address = bindAddress.getAddress().getHostAddress(); - int port = bindAddress.getPort(); + final InetSocketAddress bindAddress = (InetSocketAddress) serverChannel.localAddress(); + final String advertisedAddress; + if (bindAddress.getAddress().isAnyLocalAddress()) { + advertisedAddress = this.restAddress; + } else { + advertisedAddress = bindAddress.getAddress().getHostAddress(); + } + final int port = bindAddress.getPort(); - log.info("Rest endpoint listening at {}:{}", address, port); + log.info("Rest endpoint listening at {}:{}", advertisedAddress, port); final String protocol; @@ -197,9 +203,9 @@ protected void initChannel(SocketChannel ch) { protocol = "http://"; } - restAddress = protocol + address + ':' + port; + restBaseUrl = protocol + advertisedAddress + ':' + port; - restAddressFuture.complete(restAddress); + restAddressFuture.complete(restBaseUrl); state = State.RUNNING; @@ -238,14 +244,14 @@ public InetSocketAddress getServerAddress() { } /** - * Returns the address of the REST server endpoint. + * Returns the base URL of the REST server endpoint. * - * @return REST address of this endpoint + * @return REST base URL of this endpoint */ - public String getRestAddress() { + public String getRestBaseUrl() { synchronized (lock) { Preconditions.checkState(state != State.CREATED, "The RestServerEndpoint has not been started yet."); - return restAddress; + return restBaseUrl; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java index 35bd6ea15d77b..1fac08e53edc9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java @@ -45,6 +45,8 @@ */ public final class RestServerEndpointConfiguration { + private final String restAddress; + @Nullable private final String restBindAddress; @@ -60,6 +62,7 @@ public final class RestServerEndpointConfiguration { private final Map responseHeaders; private RestServerEndpointConfiguration( + final String restAddress, @Nullable String restBindAddress, int restBindPort, @Nullable SSLEngine sslEngine, @@ -69,12 +72,20 @@ private RestServerEndpointConfiguration( Preconditions.checkArgument(0 <= restBindPort && restBindPort < 65536, "The bing rest port " + restBindPort + " is out of range (0, 65536["); Preconditions.checkArgument(maxContentLength > 0, "maxContentLength must be positive, was: %d", maxContentLength); + this.restAddress = requireNonNull(restAddress); this.restBindAddress = restBindAddress; this.restBindPort = restBindPort; this.sslEngine = sslEngine; this.uploadDir = requireNonNull(uploadDir); this.maxContentLength = maxContentLength; - this.responseHeaders = requireNonNull(Collections.unmodifiableMap(responseHeaders)); + this.responseHeaders = Collections.unmodifiableMap(requireNonNull(responseHeaders)); + } + + /** + * @see RestOptions#REST_ADDRESS + */ + public String getRestAddress() { + return restAddress; } /** @@ -82,7 +93,7 @@ private RestServerEndpointConfiguration( * * @return address that the REST server endpoint should bind itself to */ - public String getEndpointBindAddress() { + public String getRestBindAddress() { return restBindAddress; } @@ -91,7 +102,7 @@ public String getEndpointBindAddress() { * * @return port that the REST server endpoint should listen on */ - public int getEndpointBindPort() { + public int getRestBindPort() { return restBindPort; } @@ -136,12 +147,16 @@ public Map getResponseHeaders() { */ public static RestServerEndpointConfiguration fromConfiguration(Configuration config) throws ConfigurationException { Preconditions.checkNotNull(config); - String address = config.getString(RestOptions.REST_ADDRESS); - int port = config.getInteger(RestOptions.REST_PORT); + final String restAddress = Preconditions.checkNotNull(config.getString(RestOptions.REST_ADDRESS), + "%s must be set", + RestOptions.REST_ADDRESS.key()); + + final String restBindAddress = config.getString(RestOptions.REST_BIND_ADDRESS); + final int port = config.getInteger(RestOptions.REST_PORT); SSLEngine sslEngine = null; - boolean enableSSL = config.getBoolean(SecurityOptions.SSL_ENABLED); + final boolean enableSSL = config.getBoolean(SecurityOptions.SSL_ENABLED); if (enableSSL) { try { SSLContext sslContext = SSLUtils.createSSLServerContext(config); @@ -159,14 +174,15 @@ public static RestServerEndpointConfiguration fromConfiguration(Configuration co config.getString(WebOptions.UPLOAD_DIR, config.getString(WebOptions.TMP_DIR)), "flink-web-upload-" + UUID.randomUUID()); - int maxContentLength = config.getInteger(RestOptions.REST_SERVER_MAX_CONTENT_LENGTH); + final int maxContentLength = config.getInteger(RestOptions.REST_SERVER_MAX_CONTENT_LENGTH); final Map responseHeaders = Collections.singletonMap( HttpHeaders.Names.ACCESS_CONTROL_ALLOW_ORIGIN, config.getString(WebOptions.ACCESS_CONTROL_ALLOW_ORIGIN)); return new RestServerEndpointConfiguration( - address, + restAddress, + restBindAddress, port, sslEngine, uploadDir, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index dfb2fc8591d97..50ad7eb1bceeb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -666,18 +666,18 @@ protected CompletableFuture shutDownInternal() { @Override public void grantLeadership(final UUID leaderSessionID) { - log.info("{} was granted leadership with leaderSessionID={}", getRestAddress(), leaderSessionID); + log.info("{} was granted leadership with leaderSessionID={}", getRestBaseUrl(), leaderSessionID); leaderElectionService.confirmLeaderSessionID(leaderSessionID); } @Override public void revokeLeadership() { - log.info("{} lost leadership", getRestAddress()); + log.info("{} lost leadership", getRestBaseUrl()); } @Override public String getAddress() { - return getRestAddress(); + return getRestBaseUrl(); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java index 32f3ec89cadd6..784c14158a3b4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java @@ -115,6 +115,7 @@ public class RestServerEndpointITCase extends TestLogger { public void setup() throws Exception { Configuration config = new Configuration(); config.setInteger(RestOptions.REST_PORT, 0); + config.setString(RestOptions.REST_ADDRESS, "localhost"); config.setString(WebOptions.UPLOAD_DIR, temporaryFolder.newFolder().getCanonicalPath()); config.setInteger(RestOptions.REST_SERVER_MAX_CONTENT_LENGTH, TEST_REST_MAX_CONTENT_LENGTH); config.setInteger(RestOptions.REST_CLIENT_MAX_CONTENT_LENGTH, TEST_REST_MAX_CONTENT_LENGTH); @@ -335,7 +336,7 @@ public void testMultiPartFormDataWithoutFileUpload() throws Exception { private HttpURLConnection openHttpConnectionForUpload(final String boundary) throws IOException { final HttpURLConnection connection = - (HttpURLConnection) new URL(serverEndpoint.getRestAddress() + "/upload").openConnection(); + (HttpURLConnection) new URL(serverEndpoint.getRestBaseUrl() + "/upload").openConnection(); connection.setDoOutput(true); connection.setRequestProperty("Content-Type", "multipart/form-data; boundary=" + boundary); return connection; From 7739cb71d9a3d476c66cdd29f26744e5ef3eb1b3 Mon Sep 17 00:00:00 2001 From: gyao Date: Fri, 16 Mar 2018 20:52:11 +0100 Subject: [PATCH 149/268] [FLINK-8894][REST] Set object codec for JsonGenerator used by CurrentJobIdsHandler This closes #5711. --- .../handler/legacy/CurrentJobIdsHandler.java | 2 + .../legacy/CurrentJobIdsHandlerTest.java | 62 ++++++++++++++++--- 2 files changed, 57 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandler.java index cf8a3d6d70900..ef02762e5a53a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandler.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.messages.webmonitor.JobIdsWithStatusOverview; import org.apache.flink.runtime.rest.messages.JobIdsWithStatusesOverviewHeaders; +import org.apache.flink.runtime.rest.util.RestMapperUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; @@ -70,6 +71,7 @@ public CompletableFuture handleJsonRequest( StringWriter writer = new StringWriter(); JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + gen.setCodec(RestMapperUtils.getStrictObjectMapper()); gen.writeStartObject(); gen.writeArrayFieldStart(JobIdsWithStatusOverview.FIELD_NAME_JOBS); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandlerTest.java index 0ada30d5daf08..b193122c1693d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/CurrentJobIdsHandlerTest.java @@ -18,21 +18,69 @@ package org.apache.flink.runtime.rest.handler.legacy; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.concurrent.Executors; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobmaster.JobManagerGateway; +import org.apache.flink.runtime.messages.webmonitor.JobIdsWithStatusOverview; +import org.apache.flink.util.TestLogger; -import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.util.Collections; +import java.util.concurrent.CompletableFuture; + +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.when; /** - * Tests for the CurrentJobIdsHandler. + * Tests for {@link CurrentJobIdsHandler}. */ -public class CurrentJobIdsHandlerTest { +public class CurrentJobIdsHandlerTest extends TestLogger { + + private CurrentJobIdsHandler currentJobIdsHandler; + + @Mock + private JobManagerGateway mockJobManagerGateway; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + currentJobIdsHandler = new CurrentJobIdsHandler(Executors.directExecutor(), Time.seconds(0L)); + } + @Test public void testGetPaths() { - CurrentJobIdsHandler handler = new CurrentJobIdsHandler(Executors.directExecutor(), Time.seconds(0L)); - String[] paths = handler.getPaths(); - Assert.assertEquals(1, paths.length); - Assert.assertEquals("/jobs", paths[0]); + final String[] paths = currentJobIdsHandler.getPaths(); + assertEquals(1, paths.length); + assertEquals("/jobs", paths[0]); } + + @Test + public void testHandleJsonRequest() throws Exception { + final JobID jobId = new JobID(); + final JobStatus jobStatus = JobStatus.RUNNING; + + when(mockJobManagerGateway.requestJobsOverview(any(Time.class))).thenReturn( + CompletableFuture.completedFuture(new JobIdsWithStatusOverview(Collections.singleton( + new JobIdsWithStatusOverview.JobIdWithStatus(jobId, jobStatus))))); + + final CompletableFuture jsonFuture = currentJobIdsHandler.handleJsonRequest( + Collections.emptyMap(), + Collections.emptyMap(), + mockJobManagerGateway); + + final String json = jsonFuture.get(); + + assertThat(json, containsString(jobId.toString())); + assertThat(json, containsString(jobStatus.name())); + } + } From 341a3fd8e20a4645dd8326f5c290314c2344f728 Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 15 Mar 2018 16:24:35 +0800 Subject: [PATCH 150/268] [FLINK-8915] CheckpointingStatisticsHandler fails to return PendingCheckpointStats This closes #5703. --- .../checkpoints/CheckpointStatistics.java | 77 ++++++++++++++++++- .../CheckpointingStatisticsTest.java | 16 +++- 2 files changed, 90 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java index 333c0167bee1c..f8aeb266aaaca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointStatsStatus; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats; import org.apache.flink.runtime.checkpoint.FailedCheckpointStats; +import org.apache.flink.runtime.checkpoint.PendingCheckpointStats; import org.apache.flink.runtime.checkpoint.TaskStateStats; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.rest.messages.ResponseBody; @@ -50,7 +51,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "@class") @JsonSubTypes({ @JsonSubTypes.Type(value = CheckpointStatistics.CompletedCheckpointStatistics.class, name = "completed"), - @JsonSubTypes.Type(value = CheckpointStatistics.FailedCheckpointStatistics.class, name = "failed")}) + @JsonSubTypes.Type(value = CheckpointStatistics.FailedCheckpointStatistics.class, name = "failed"), + @JsonSubTypes.Type(value = CheckpointStatistics.PendingCheckpointStatistics.class, name = "in_progress")}) public class CheckpointStatistics implements ResponseBody { public static final String FIELD_NAME_ID = "id"; @@ -272,8 +274,25 @@ public static CheckpointStatistics generateCheckpointStatistics(AbstractCheckpoi checkpointStatisticsPerTask, failedCheckpointStats.getFailureTimestamp(), failedCheckpointStats.getFailureMessage()); + } else if (checkpointStats instanceof PendingCheckpointStats) { + final PendingCheckpointStats pendingCheckpointStats = ((PendingCheckpointStats) checkpointStats); + + return new CheckpointStatistics.PendingCheckpointStatistics( + pendingCheckpointStats.getCheckpointId(), + pendingCheckpointStats.getStatus(), + pendingCheckpointStats.getProperties().isSavepoint(), + pendingCheckpointStats.getTriggerTimestamp(), + pendingCheckpointStats.getLatestAckTimestamp(), + pendingCheckpointStats.getStateSize(), + pendingCheckpointStats.getEndToEndDuration(), + pendingCheckpointStats.getAlignmentBuffered(), + pendingCheckpointStats.getNumberOfSubtasks(), + pendingCheckpointStats.getNumberOfAcknowledgedSubtasks(), + checkpointStatisticsPerTask + ); } else { - throw new IllegalArgumentException("Given checkpoint stats object of type " + checkpointStats.getClass().getName() + " cannot be converted."); + throw new IllegalArgumentException("Given checkpoint stats object of type " + + checkpointStats.getClass().getName() + " cannot be converted."); } } @@ -438,4 +457,58 @@ public int hashCode() { return Objects.hash(super.hashCode(), failureTimestamp, failureMessage); } } + + /** + * Statistics for a pending checkpoint. + */ + public static final class PendingCheckpointStatistics extends CheckpointStatistics { + + @JsonCreator + public PendingCheckpointStatistics( + @JsonProperty(FIELD_NAME_ID) long id, + @JsonProperty(FIELD_NAME_STATUS) CheckpointStatsStatus status, + @JsonProperty(FIELD_NAME_IS_SAVEPOINT) boolean savepoint, + @JsonProperty(FIELD_NAME_TRIGGER_TIMESTAMP) long triggerTimestamp, + @JsonProperty(FIELD_NAME_LATEST_ACK_TIMESTAMP) long latestAckTimestamp, + @JsonProperty(FIELD_NAME_STATE_SIZE) long stateSize, + @JsonProperty(FIELD_NAME_DURATION) long duration, + @JsonProperty(FIELD_NAME_ALIGNMENT_BUFFERED) long alignmentBuffered, + @JsonProperty(FIELD_NAME_NUM_SUBTASKS) int numSubtasks, + @JsonProperty(FIELD_NAME_NUM_ACK_SUBTASKS) int numAckSubtasks, + @JsonDeserialize(keyUsing = JobVertexIDKeyDeserializer.class) @JsonProperty(FIELD_NAME_TASKS) Map checkpointingStatisticsPerTask) { + super( + id, + status, + savepoint, + triggerTimestamp, + latestAckTimestamp, + stateSize, + duration, + alignmentBuffered, + numSubtasks, + numAckSubtasks, + checkpointingStatisticsPerTask); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + + return true; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode()); + } + + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatisticsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatisticsTest.java index 562418e64077c..8f25a59fd282e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatisticsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointingStatisticsTest.java @@ -122,6 +122,20 @@ protected CheckpointingStatistics getTestResponseInstance() throws Exception { true, "foobar"); + CheckpointStatistics.PendingCheckpointStatistics pending = new CheckpointStatistics.PendingCheckpointStatistics( + 5L, + CheckpointStatsStatus.IN_PROGRESS, + false, + 42L, + 41L, + 1337L, + 1L, + 0L, + 10, + 10, + Collections.emptyMap() + ); + final CheckpointingStatistics.LatestCheckpoints latestCheckpoints = new CheckpointingStatistics.LatestCheckpoints( completed, savepoint, @@ -132,6 +146,6 @@ protected CheckpointingStatistics getTestResponseInstance() throws Exception { counts, summary, latestCheckpoints, - Arrays.asList(completed, savepoint, failed)); + Arrays.asList(completed, savepoint, failed, pending)); } } From 76b0eb670921065ea2348cebbb62779d7fba6351 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Fri, 9 Mar 2018 11:05:51 +0100 Subject: [PATCH 151/268] [FLINK-8905][rest][client] fix RestClusterClient#getMaxSlots() returning 0 --- .../main/java/org/apache/flink/client/cli/CliFrontend.java | 3 ++- .../org/apache/flink/client/program/ClusterClient.java | 7 ++++++- .../org/apache/flink/client/program/MiniClusterClient.java | 2 +- .../flink/client/program/StandaloneClusterClient.java | 2 +- .../flink/client/program/rest/RestClusterClient.java | 2 +- .../main/java/org/apache/flink/yarn/YarnClusterClient.java | 2 +- 6 files changed, 12 insertions(+), 6 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index 06131dc6836b0..d636ef7c9615e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -86,6 +86,7 @@ import static org.apache.flink.client.cli.CliFrontendParser.HELP_OPTION; import static org.apache.flink.client.cli.CliFrontendParser.MODIFY_PARALLELISM_OPTION; +import static org.apache.flink.client.program.ClusterClient.MAX_SLOTS_UNKNOWN; /** * Implementation of a simple command line frontend for executing programs. @@ -262,7 +263,7 @@ private void runProgram( int userParallelism = runOptions.getParallelism(); LOG.debug("User parallelism is set to {}", userParallelism); - if (client.getMaxSlots() != -1 && userParallelism == -1) { + if (client.getMaxSlots() != MAX_SLOTS_UNKNOWN && userParallelism == -1) { logAndSysout("Using the parallelism provided by the remote cluster (" + client.getMaxSlots() + "). " + "To use another parallelism, set it at the ./bin/flink client."); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index 1a783fc221384..b0c50e59d7705 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -133,6 +133,11 @@ public abstract class ClusterClient { /** Switch for blocking/detached job submission of the client. */ private boolean detachedJobSubmission = false; + /** + * Value returned by {@link #getMaxSlots()} if the number of maximum slots is unknown. + */ + public static final int MAX_SLOTS_UNKNOWN = -1; + // ------------------------------------------------------------------------ // Construction // ------------------------------------------------------------------------ @@ -1000,7 +1005,7 @@ public Configuration getFlinkConfiguration() { /** * The client may define an upper limit on the number of slots to use. - * @return -1 if unknown + * @return -1 ({@link #MAX_SLOTS_UNKNOWN}) if unknown */ public abstract int getMaxSlots(); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index f0a7631023ea4..86e9279575050 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -181,7 +181,7 @@ public List getNewMessages() { @Override public int getMaxSlots() { - return 0; + return MAX_SLOTS_UNKNOWN; } @Override diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java index 1c9c690710ee1..e502add468c76 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/StandaloneClusterClient.java @@ -98,7 +98,7 @@ public StandaloneClusterId getClusterId() { @Override public int getMaxSlots() { - return -1; + return MAX_SLOTS_UNKNOWN; } @Override diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 8cf0d2c7eb200..5558461dbf609 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -567,7 +567,7 @@ public List getNewMessages() { @Override public int getMaxSlots() { - return 0; + return MAX_SLOTS_UNKNOWN; } //------------------------------------------------------------------------- diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java index e0010c769ae0e..29ece26719ea7 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterClient.java @@ -139,7 +139,7 @@ public org.apache.flink.configuration.Configuration getFlinkConfiguration() { public int getMaxSlots() { // TODO: this should be retrieved from the running Flink cluster int maxSlots = numberTaskManagers * slotsPerTaskManager; - return maxSlots > 0 ? maxSlots : -1; + return maxSlots > 0 ? maxSlots : MAX_SLOTS_UNKNOWN; } @Override From 0ba8ed6ba8b2f17de79e05207dc14226504b8be9 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 8 Mar 2018 11:07:08 +0100 Subject: [PATCH 152/268] [FLINK-8906][flip6][tests] also test Flip6DefaultCLI in org.apache.flink.client.cli tests This closes #5671. --- .../client/cli/CliFrontendCancelTest.java | 19 +++--- .../flink/client/cli/CliFrontendInfoTest.java | 19 +++--- .../flink/client/cli/CliFrontendListTest.java | 7 +-- .../client/cli/CliFrontendModifyTest.java | 10 +-- .../flink/client/cli/CliFrontendRunTest.java | 50 +++++++++------ .../client/cli/CliFrontendSavepointTest.java | 15 +++-- .../flink/client/cli/CliFrontendStopTest.java | 11 ++-- .../flink/client/cli/CliFrontendTestBase.java | 61 +++++++++++++++++++ .../flink/client/cli/DefaultCLITest.java | 24 +++++--- 9 files changed, 144 insertions(+), 72 deletions(-) create mode 100644 flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java index 837c56408699c..638150a2e72a8 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendCancelTest.java @@ -22,7 +22,6 @@ import org.apache.flink.client.cli.util.MockedCliFrontend; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; -import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -40,7 +39,7 @@ /** * Tests for the CANCEL command. */ -public class CliFrontendCancelTest extends TestLogger { +public class CliFrontendCancelTest extends CliFrontendTestBase { @BeforeClass public static void init() { @@ -69,20 +68,20 @@ public void testCancel() throws Exception { @Test(expected = CliArgsException.class) public void testMissingJobId() throws Exception { String[] parameters = {}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.cancel(parameters); } @Test(expected = CliArgsException.class) public void testUnrecognizedOption() throws Exception { String[] parameters = {"-v", "-l"}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.cancel(parameters); } @@ -122,10 +121,10 @@ public void testCancelWithSavepoint() throws Exception { public void testCancelWithSavepointWithoutJobId() throws Exception { // Cancel with savepoint (with target directory), but no job ID String[] parameters = { "-s", "targetDirectory" }; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.cancel(parameters); } @@ -133,10 +132,10 @@ public void testCancelWithSavepointWithoutJobId() throws Exception { public void testCancelWithSavepointWithoutParameters() throws Exception { // Cancel with savepoint (no target directory) and no job ID String[] parameters = { "-s" }; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.cancel(parameters); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java index c284c6141b7a3..47c799ee98120 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendInfoTest.java @@ -19,7 +19,6 @@ package org.apache.flink.client.cli; import org.apache.flink.configuration.Configuration; -import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -33,7 +32,7 @@ /** * Tests for the "info" command. */ -public class CliFrontendInfoTest extends TestLogger { +public class CliFrontendInfoTest extends CliFrontendTestBase { private static PrintStream stdOut; private static PrintStream capture; @@ -42,20 +41,20 @@ public class CliFrontendInfoTest extends TestLogger { @Test(expected = CliArgsException.class) public void testMissingOption() throws Exception { String[] parameters = {}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.cancel(parameters); } @Test(expected = CliArgsException.class) public void testUnrecognizedOption() throws Exception { String[] parameters = {"-v", "-l"}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.cancel(parameters); } @@ -65,10 +64,10 @@ public void testShowExecutionPlan() throws Exception { try { String[] parameters = new String[]{CliFrontendTestUtils.getTestJarPath(), "-f", "true"}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.info(parameters); assertTrue(buffer.toString().contains("\"parallelism\": \"1\"")); } @@ -82,10 +81,10 @@ public void testShowExecutionPlanWithParallelism() { replaceStdOut(); try { String[] parameters = {"-p", "17", CliFrontendTestUtils.getTestJarPath()}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.info(parameters); assertTrue(buffer.toString().contains("\"parallelism\": \"17\"")); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java index 42399cb65f09b..a8a7f006bda3d 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendListTest.java @@ -21,7 +21,6 @@ import org.apache.flink.client.cli.util.MockedCliFrontend; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; -import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -38,7 +37,7 @@ /** * Tests for the LIST command. */ -public class CliFrontendListTest extends TestLogger { +public class CliFrontendListTest extends CliFrontendTestBase { @BeforeClass public static void init() { @@ -66,10 +65,10 @@ public void testList() throws Exception { @Test(expected = CliArgsException.class) public void testUnrecognizedOption() throws Exception { String[] parameters = {"-v", "-k"}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.list(parameters); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendModifyTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendModifyTest.java index 00d52419537f3..a2d6c48f01d99 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendModifyTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendModifyTest.java @@ -25,7 +25,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.util.TestLogger; import org.hamcrest.Matchers; import org.junit.Test; @@ -38,7 +37,7 @@ /** * Tests for the modify command. */ -public class CliFrontendModifyTest extends TestLogger { +public class CliFrontendModifyTest extends CliFrontendTestBase { @Test public void testModifyJob() throws Exception { @@ -106,7 +105,7 @@ public void testUnparsableJobId() throws Exception { private Tuple2 callModify(String[] args) throws Exception { final CompletableFuture> rescaleJobFuture = new CompletableFuture<>(); - final TestingClusterClient clusterClient = new TestingClusterClient(rescaleJobFuture); + final TestingClusterClient clusterClient = new TestingClusterClient(rescaleJobFuture, getConfiguration()); final MockedCliFrontend cliFrontend = new MockedCliFrontend(clusterClient); cliFrontend.modify(args); @@ -120,8 +119,9 @@ private static final class TestingClusterClient extends StandaloneClusterClient private final CompletableFuture> rescaleJobFuture; - public TestingClusterClient(CompletableFuture> rescaleJobFuture) throws Exception { - super(new Configuration(), new TestingHighAvailabilityServices(), false); + TestingClusterClient( + CompletableFuture> rescaleJobFuture, Configuration configuration) { + super(configuration, new TestingHighAvailabilityServices(), false); this.rescaleJobFuture = rescaleJobFuture; } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java index c7789a89ac5c2..efa6a3964a786 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendRunTest.java @@ -21,9 +21,7 @@ import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -39,7 +37,7 @@ /** * Tests for the RUN command. */ -public class CliFrontendRunTest extends TestLogger { +public class CliFrontendRunTest extends CliFrontendTestBase { @BeforeClass public static void init() { @@ -53,33 +51,29 @@ public static void shutdown() { @Test public void testRun() throws Exception { - final Configuration configuration = GlobalConfiguration.loadConfiguration(CliFrontendTestUtils.getConfigDir()); + final Configuration configuration = getConfiguration(); // test without parallelism { String[] parameters = {"-v", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 1, true, false); - testFrontend.run(parameters); + verifyCliFrontend(getCli(configuration), parameters, 1, true, false); } // test configure parallelism { String[] parameters = {"-v", "-p", "42", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 42, true, false); - testFrontend.run(parameters); + verifyCliFrontend(getCli(configuration), parameters, 42, true, false); } // test configure sysout logging { String[] parameters = {"-p", "2", "-q", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 2, false, false); - testFrontend.run(parameters); + verifyCliFrontend(getCli(configuration), parameters, 2, false, false); } // test detached mode { String[] parameters = {"-p", "2", "-d", getTestJarPath()}; - RunTestingCliFrontend testFrontend = new RunTestingCliFrontend(configuration, 2, true, true); - testFrontend.run(parameters); + verifyCliFrontend(getCli(configuration), parameters, 2, true, true); } // test configure savepoint path (no ignore flag) @@ -119,10 +113,10 @@ public void testRun() throws Exception { public void testUnrecognizedOption() throws Exception { // test unrecognized option String[] parameters = {"-v", "-l", "-a", "some", "program", "arguments"}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.run(parameters); } @@ -130,10 +124,10 @@ public void testUnrecognizedOption() throws Exception { public void testInvalidParallelismOption() throws Exception { // test configure parallelism with non integer value String[] parameters = {"-v", "-p", "text", getTestJarPath()}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.run(parameters); } @@ -144,22 +138,38 @@ public void testParallelismWithOverflow() throws Exception { Configuration configuration = new Configuration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.run(parameters); } // -------------------------------------------------------------------------------------------- + private static void verifyCliFrontend( + AbstractCustomCommandLine cli, + String[] parameters, + int expectedParallelism, + boolean logging, + boolean isDetached) throws Exception { + RunTestingCliFrontend testFrontend = + new RunTestingCliFrontend(cli, expectedParallelism, logging, + isDetached); + testFrontend.run(parameters); // verifies the expected values (see below) + } + private static final class RunTestingCliFrontend extends CliFrontend { private final int expectedParallelism; private final boolean sysoutLogging; private final boolean isDetached; - public RunTestingCliFrontend(Configuration configuration, int expectedParallelism, boolean logging, boolean isDetached) throws Exception { + private RunTestingCliFrontend( + AbstractCustomCommandLine cli, + int expectedParallelism, + boolean logging, + boolean isDetached) throws Exception { super( - configuration, - Collections.singletonList(new DefaultCLI(configuration))); + cli.getConfiguration(), + Collections.singletonList(cli)); this.expectedParallelism = expectedParallelism; this.sysoutLogging = logging; this.isDetached = isDetached; diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java index f4c66eb08c204..3195a6baf6dc1 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java @@ -30,7 +30,6 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; -import org.apache.flink.util.TestLogger; import org.hamcrest.Matchers; import org.junit.Rule; @@ -61,7 +60,7 @@ /** * Tests for the SAVEPOINT command. */ -public class CliFrontendSavepointTest extends TestLogger { +public class CliFrontendSavepointTest extends CliFrontendTestBase { private static PrintStream stdOut; private static PrintStream stdErr; @@ -137,7 +136,7 @@ public void testTriggerSavepointFailureIllegalJobID() throws Exception { try { CliFrontend frontend = new MockedCliFrontend(new StandaloneClusterClient( - new Configuration(), + getConfiguration(), new TestingHighAvailabilityServices(), false)); @@ -197,7 +196,7 @@ public void testDisposeSavepointSuccess() throws Exception { String savepointPath = "expectedSavepointPath"; ClusterClient clusterClient = new DisposeSavepointClusterClient( - (String path, Time timeout) -> CompletableFuture.completedFuture(Acknowledge.get())); + (String path, Time timeout) -> CompletableFuture.completedFuture(Acknowledge.get()), getConfiguration()); try { @@ -229,7 +228,7 @@ public void testDisposeWithJar() throws Exception { (String savepointPath, Time timeout) -> { disposeSavepointFuture.complete(savepointPath); return CompletableFuture.completedFuture(Acknowledge.get()); - }); + }, getConfiguration()); try { CliFrontend frontend = new MockedCliFrontend(clusterClient); @@ -261,7 +260,7 @@ public void testDisposeSavepointFailure() throws Exception { Exception testException = new Exception("expectedTestException"); - DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> FutureUtils.completedExceptionally(testException)); + DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> FutureUtils.completedExceptionally(testException), getConfiguration()); try { CliFrontend frontend = new MockedCliFrontend(clusterClient); @@ -288,8 +287,8 @@ private static final class DisposeSavepointClusterClient extends StandaloneClust private final BiFunction> disposeSavepointFunction; - DisposeSavepointClusterClient(BiFunction> disposeSavepointFunction) throws Exception { - super(new Configuration(), new TestingHighAvailabilityServices(), false); + DisposeSavepointClusterClient(BiFunction> disposeSavepointFunction, Configuration configuration) { + super(configuration, new TestingHighAvailabilityServices(), false); this.disposeSavepointFunction = Preconditions.checkNotNull(disposeSavepointFunction); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java index ec4ccdca7d1df..23bed005317c6 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopTest.java @@ -24,7 +24,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; -import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -45,7 +44,7 @@ /** * Tests for the STOP command. */ -public class CliFrontendStopTest extends TestLogger { +public class CliFrontendStopTest extends CliFrontendTestBase { @BeforeClass public static void setup() { @@ -76,10 +75,10 @@ public void testStop() throws Exception { public void testUnrecognizedOption() throws Exception { // test unrecognized option String[] parameters = { "-v", "-l" }; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.stop(parameters); } @@ -87,10 +86,10 @@ public void testUnrecognizedOption() throws Exception { public void testMissingJobId() throws Exception { // test missing job id String[] parameters = {}; - Configuration configuration = new Configuration(); + Configuration configuration = getConfiguration(); CliFrontend testFrontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(getCli(configuration))); testFrontend.stop(parameters); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java new file mode 100644 index 0000000000000..e2463f214c430 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java @@ -0,0 +1,61 @@ +/* + * 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.client.cli; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.util.TestLogger; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.List; + +/** + * Base test class for {@link CliFrontend} tests that wraps the Flip-6 vs. non-Flip-6 modes. + */ +@RunWith(Parameterized.class) +public abstract class CliFrontendTestBase extends TestLogger { + @Parameterized.Parameter + public String mode; + + @Parameterized.Parameters(name = "Mode = {0}") + public static List parameters() { + return Arrays.asList(CoreOptions.OLD_MODE, CoreOptions.FLIP6_MODE); + } + + protected Configuration getConfiguration() { + final Configuration configuration = GlobalConfiguration + .loadConfiguration(CliFrontendTestUtils.getConfigDir()); + configuration.setString(CoreOptions.MODE, mode); + return configuration; + } + + static AbstractCustomCommandLine getCli(Configuration configuration) { + switch (configuration.getString(CoreOptions.MODE)) { + case CoreOptions.OLD_MODE: + return new DefaultCLI(configuration); + case CoreOptions.FLIP6_MODE: + return new Flip6DefaultCLI(configuration); + } + throw new IllegalStateException(); + } +} diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java index d89e988aebb6d..8402b7e112099 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java @@ -18,12 +18,12 @@ package org.apache.flink.client.cli; -import org.apache.flink.client.deployment.StandaloneClusterDescriptor; +import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.util.LeaderConnectionInfo; -import org.apache.flink.util.TestLogger; import org.apache.commons.cli.CommandLine; import org.hamcrest.Matchers; @@ -36,7 +36,7 @@ /** * Tests for the {@link DefaultCLI}. */ -public class DefaultCLITest extends TestLogger { +public class DefaultCLITest extends CliFrontendTestBase { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -47,7 +47,7 @@ public class DefaultCLITest extends TestLogger { */ @Test public void testConfigurationPassing() throws Exception { - final Configuration configuration = new Configuration(); + final Configuration configuration = getConfiguration(); final String localhost = "localhost"; final int port = 1234; @@ -55,13 +55,16 @@ public void testConfigurationPassing() throws Exception { configuration.setString(JobManagerOptions.ADDRESS, localhost); configuration.setInteger(JobManagerOptions.PORT, port); - final DefaultCLI defaultCLI = new DefaultCLI(configuration); + @SuppressWarnings("unchecked") + final AbstractCustomCommandLine defaultCLI = + (AbstractCustomCommandLine) getCli(configuration); final String[] args = {}; CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); - final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine); + final ClusterDescriptor clusterDescriptor = + defaultCLI.createClusterDescriptor(commandLine); final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine)); @@ -78,12 +81,14 @@ public void testConfigurationPassing() throws Exception { public void testManualConfigurationOverride() throws Exception { final String localhost = "localhost"; final int port = 1234; - final Configuration configuration = new Configuration(); + final Configuration configuration = getConfiguration(); configuration.setString(JobManagerOptions.ADDRESS, localhost); configuration.setInteger(JobManagerOptions.PORT, port); - final DefaultCLI defaultCLI = new DefaultCLI(configuration); + @SuppressWarnings("unchecked") + final AbstractCustomCommandLine defaultCLI = + (AbstractCustomCommandLine) getCli(configuration); final String manualHostname = "123.123.123.123"; final int manualPort = 4321; @@ -91,7 +96,8 @@ public void testManualConfigurationOverride() throws Exception { CommandLine commandLine = defaultCLI.parseCommandLineOptions(args, false); - final StandaloneClusterDescriptor clusterDescriptor = defaultCLI.createClusterDescriptor(commandLine); + final ClusterDescriptor clusterDescriptor = + defaultCLI.createClusterDescriptor(commandLine); final ClusterClient clusterClient = clusterDescriptor.retrieve(defaultCLI.getClusterId(commandLine)); From d5b712715fc89a0f56f29be8668e73ba4630f4d5 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 16 Mar 2018 15:56:07 +0100 Subject: [PATCH 153/268] [FLINK-8948][runtime] Fix IllegalStateException when closing StreamTask This closes #5710. --- .../runtime/io/network/buffer/BufferBuilder.java | 5 +++-- .../buffer/BufferBuilderAndConsumerTest.java | 16 ++++++++++++---- 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java index 63b60d2b8a080..305f1842911c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferBuilder.java @@ -99,10 +99,11 @@ public void commit() { * Mark this {@link BufferBuilder} and associated {@link BufferConsumer} as finished - no new data writes will be * allowed. * + *

    This method should be idempotent to handle failures and task interruptions. Check FLINK-8948 for more details. + * * @return number of written bytes. */ public int finish() { - checkState(!isFinished()); positionMarker.markFinished(); commit(); return getWrittenBytes(); @@ -125,7 +126,7 @@ public int getMaxCapacity() { return memorySegment.size(); } - public int getWrittenBytes() { + private int getWrittenBytes() { return positionMarker.getCached(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java index edf2bfe319ead..b5d9da0f1aa3e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferBuilderAndConsumerTest.java @@ -202,22 +202,30 @@ private static void testIsFinished(int writes) { for (int i = 0; i < writes; i++) { assertEquals(Integer.BYTES, bufferBuilder.appendAndCommit(toByteBuffer(42))); } + int expectedWrittenBytes = writes * Integer.BYTES; assertFalse(bufferBuilder.isFinished()); assertFalse(bufferConsumer.isFinished()); + assertEquals(0, bufferConsumer.getWrittenBytes()); bufferConsumer.build(); - assertFalse(bufferBuilder.isFinished()); assertFalse(bufferConsumer.isFinished()); + assertEquals(expectedWrittenBytes, bufferConsumer.getWrittenBytes()); - bufferBuilder.finish(); - + int actualWrittenBytes = bufferBuilder.finish(); + assertEquals(expectedWrittenBytes, actualWrittenBytes); assertTrue(bufferBuilder.isFinished()); assertFalse(bufferConsumer.isFinished()); + assertEquals(expectedWrittenBytes, bufferConsumer.getWrittenBytes()); - bufferConsumer.build(); + actualWrittenBytes = bufferBuilder.finish(); + assertEquals(expectedWrittenBytes, actualWrittenBytes); + assertTrue(bufferBuilder.isFinished()); + assertFalse(bufferConsumer.isFinished()); + assertEquals(expectedWrittenBytes, bufferConsumer.getWrittenBytes()); + assertEquals(0, bufferConsumer.build().getSize()); assertTrue(bufferConsumer.isFinished()); } From 92ed0b2aea93839f6cb867db8885bbdd2e298870 Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Wed, 14 Mar 2018 16:22:44 +0800 Subject: [PATCH 154/268] [hotfix][javadocs] Minor javadoc fixes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This close sƤ5696 --- .../apache/flink/runtime/jobmaster/slotpool/SlotProvider.java | 3 ++- .../org/apache/flink/runtime/state/heap/AbstractHeapState.java | 3 +-- .../apache/flink/runtime/state/heap/HeapAggregatingState.java | 3 +-- .../flink/contrib/streaming/state/RocksDBAggregatingState.java | 2 +- 4 files changed, 5 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java index 1653138949b3f..91c0372dcb387 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotProvider.java @@ -46,9 +46,10 @@ public interface SlotProvider { /** * Allocating slot with specific requirement. * + * @param slotRequestId identifying the slot request * @param task The task to allocate the slot for * @param allowQueued Whether allow the task be queued if we do not have enough resource - * @param preferredLocations preferred locations for the slot allocation + * @param slotProfile profile of the requested slot * @param timeout after which the allocation fails with a timeout exception * @return The future of the allocation */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java index 66360e4e9fffb..7f629ae9e031d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/AbstractHeapState.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.state.heap; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.State; import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -29,7 +28,7 @@ import org.apache.flink.util.Preconditions; /** - * Base class for partitioned {@link ListState} implementations that are backed by a regular + * Base class for partitioned {@link State} implementations that are backed by a regular * heap hash map. The concrete implementations define how the state is checkpointed. * * @param The type of the key. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java index 3fa8cd4781dea..6dd5cec878a3a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapAggregatingState.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.state.AggregatingState; import org.apache.flink.api.common.state.AggregatingStateDescriptor; -import org.apache.flink.api.common.state.ReducingState; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.state.StateTransformationFunction; import org.apache.flink.runtime.state.internal.InternalAggregatingState; @@ -30,7 +29,7 @@ import java.io.IOException; /** - * Heap-backed partitioned {@link ReducingState} that is + * Heap-backed partitioned {@link AggregatingState} that is * snapshotted into files. * * @param The type of the key. diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java index f2d1d865bc056..4dfc77228f290 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBAggregatingState.java @@ -53,7 +53,7 @@ public class RocksDBAggregatingState private final AggregateFunction aggFunction; /** - * Creates a new {@code RocksDBFoldingState}. + * Creates a new {@code RocksDBAggregatingState}. * * @param namespaceSerializer * The serializer for the namespace. From 74525afca7a741b1d26b587a35ddc4ac30214d28 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Mar 2018 11:05:12 +0100 Subject: [PATCH 155/268] [FLINK-8935][tests] Implement MiniClusterClient#listJobs --- .../flink/client/program/MiniClusterClient.java | 2 +- .../flink/runtime/minicluster/MiniCluster.java | 16 ++++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index 86e9279575050..961604f3a865d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -125,7 +125,7 @@ public CompletableFuture disposeSavepoint(String savepointPath, Tim @Override public CompletableFuture> listJobs() throws Exception { - throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + return guardWithSingleRetry(miniCluster::listJobs, scheduledExecutor); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index dfe30afda5eb6..21b89ec1fd1b0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.clusterframework.FlinkResourceManager; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -87,6 +88,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; @@ -473,6 +475,20 @@ public CompletableFuture closeAsync() { // Accessing jobs // ------------------------------------------------------------------------ + public CompletableFuture> listJobs() { + try { + return getDispatcherGateway().requestMultipleJobDetails(rpcTimeout) + .thenApply(jobs -> jobs.getJobs().stream() + .map(details -> new JobStatusMessage(details.getJobId(), details.getJobName(), details.getStatus(), details.getStartTime())) + .collect(Collectors.toList())); + } catch (LeaderRetrievalException | InterruptedException e) { + return FutureUtils.completedExceptionally( + new FlinkException( + "Could not retrieve job list.", + e)); + } + } + public CompletableFuture getJobStatus(JobID jobId) { try { return getDispatcherGateway().requestJobStatus(jobId, rpcTimeout); From d86a2f0fbe3ca55fce18e6b319419eca04739a6a Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Mar 2018 11:05:42 +0100 Subject: [PATCH 156/268] [FLINK-8935][tests] Implement MiniClusterClient#getAccumulators --- .../flink/client/program/MiniClusterClient.java | 13 +++++++++++-- .../flink/runtime/minicluster/MiniCluster.java | 12 ++++++++++++ 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index 961604f3a865d..9c8742377959e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalException; @@ -40,6 +41,7 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.SerializedValue; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -47,6 +49,7 @@ import java.net.URL; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -130,12 +133,18 @@ public CompletableFuture> listJobs() throws Excepti @Override public Map getAccumulators(JobID jobID) throws Exception { - throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + return getAccumulators(jobID, ClassLoader.getSystemClassLoader()); } @Override public Map getAccumulators(JobID jobID, ClassLoader loader) throws Exception { - throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + AccessExecutionGraph executionGraph = guardWithSingleRetry(() -> miniCluster.getExecutionGraph(jobID), scheduledExecutor).get(); + Map> accumulatorsSerialized = executionGraph.getAccumulatorsSerialized(); + Map result = new HashMap<>(accumulatorsSerialized.size()); + for (Map.Entry> acc : accumulatorsSerialized.entrySet()) { + result.put(acc.getKey(), acc.getValue().deserializeValue(loader)); + } + return result; } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 21b89ec1fd1b0..e958005e47b82 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.dispatcher.MemoryArchivedExecutionGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; import org.apache.flink.runtime.entrypoint.ClusterInformation; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; @@ -522,6 +523,17 @@ public CompletableFuture triggerSavepoint(JobID jobId, String targetDire } } + public CompletableFuture getExecutionGraph(JobID jobId) { + try { + return getDispatcherGateway().requestJob(jobId, rpcTimeout); + } catch (LeaderRetrievalException | InterruptedException e) { + return FutureUtils.completedExceptionally( + new FlinkException( + String.format("Could not retrieve job job %s.", jobId), + e)); + } + } + // ------------------------------------------------------------------------ // running jobs // ------------------------------------------------------------------------ From 29335304212e270e7f994cd430ff25c64b395cb5 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 6 Mar 2018 13:26:59 +0100 Subject: [PATCH 157/268] [FLINK-8935][tests] Implement MiniClusterClient#triggerSavepoint --- .../java/org/apache/flink/client/program/MiniClusterClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index 9c8742377959e..4354267bab993 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -118,7 +118,7 @@ public void stop(JobID jobId) throws Exception { @Override public CompletableFuture triggerSavepoint(JobID jobId, @Nullable String savepointDirectory) throws FlinkException { - throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + return guardWithSingleRetry(() -> miniCluster.triggerSavepoint(jobId, savepointDirectory, false), scheduledExecutor); } @Override From 62d927371d508bc00ab2bc55965387abafb38daa Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 7 Mar 2018 13:02:27 +0100 Subject: [PATCH 158/268] [FLINK-8935][tests] Implement MiniClusterClient#stop This closes #5690. --- .../flink/client/program/MiniClusterClient.java | 2 +- .../apache/flink/runtime/minicluster/MiniCluster.java | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index 4354267bab993..276df62f7233d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -113,7 +113,7 @@ public String cancelWithSavepoint(JobID jobId, @Nullable String savepointDirecto @Override public void stop(JobID jobId) throws Exception { - throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + guardWithSingleRetry(() -> miniCluster.stopJob(jobId), scheduledExecutor).get(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index e958005e47b82..bc75a547b55f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -512,6 +512,17 @@ public CompletableFuture cancelJob(JobID jobId) { } } + public CompletableFuture stopJob(JobID jobId) { + try { + return getDispatcherGateway().stopJob(jobId, rpcTimeout); + } catch (LeaderRetrievalException | InterruptedException e) { + return FutureUtils.completedExceptionally( + new FlinkException( + String.format("Could not stop job %s.", jobId), + e)); + } + } + public CompletableFuture triggerSavepoint(JobID jobId, String targetDirectory, boolean cancelJob) { try { return getDispatcherGateway().triggerSavepoint(jobId, targetDirectory, cancelJob, rpcTimeout); From 9442373e1b66f119e189386893426e886ad94cce Mon Sep 17 00:00:00 2001 From: vinoyang Date: Sun, 4 Mar 2018 17:01:35 +0800 Subject: [PATCH 159/268] [FLINK-8830][YARN] YarnResourceManager throws NullPointerException --- .../java/org/apache/flink/yarn/Utils.java | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index ff2478ede1415..79a670315df29 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -496,18 +496,20 @@ static ContainerLaunchContext createTaskExecutorContext( // NOTE: must read the tokens from the local file, not from the UGI context, because if UGI is login // using Kerberos keytabs, there is no HDFS delegation token in the UGI context. String fileLocation = System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); - Method readTokenStorageFileMethod = Credentials.class.getMethod( - "readTokenStorageFile", File.class, org.apache.hadoop.conf.Configuration.class); - - Credentials cred = - (Credentials) readTokenStorageFileMethod.invoke( - null, - new File(fileLocation), - HadoopUtils.getHadoopConfiguration(flinkConfig)); - - cred.writeTokenStorageToStream(dob); - ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); - ctx.setTokens(securityTokens); + if (fileLocation != null) { + Method readTokenStorageFileMethod = Credentials.class.getMethod( + "readTokenStorageFile", File.class, org.apache.hadoop.conf.Configuration.class); + + Credentials cred = + (Credentials) readTokenStorageFileMethod.invoke( + null, + new File(fileLocation), + HadoopUtils.getHadoopConfiguration(flinkConfig)); + + cred.writeTokenStorageToStream(dob); + ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); + ctx.setTokens(securityTokens); + } } catch (Throwable t) { log.error("Getting current user info failed when trying to launch the container", t); From e4906a7aa8494206f3cd2618be5cd04ec98ab014 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 19 Mar 2018 18:06:12 +0100 Subject: [PATCH 160/268] [FLINK-8830] [yarn] Log reading of Hadoop's token file This closes #5629. --- .../java/org/apache/flink/yarn/Utils.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index 79a670315df29..b9f7fac297862 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -489,14 +489,15 @@ static ContainerLaunchContext createTaskExecutorContext( ctx.setEnvironment(containerEnv); - try (DataOutputBuffer dob = new DataOutputBuffer()) { - log.debug("Adding security tokens to Task Executor Container launch Context...."); + // For TaskManager YARN container context, read the tokens from the jobmanager yarn container local file. + // NOTE: must read the tokens from the local file, not from the UGI context, because if UGI is login + // using Kerberos keytabs, there is no HDFS delegation token in the UGI context. + final String fileLocation = System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); + + if (fileLocation != null) { + log.debug("Adding security tokens to TaskExecutor's container launch context."); - // For TaskManager YARN container context, read the tokens from the jobmanager yarn container local flie. - // NOTE: must read the tokens from the local file, not from the UGI context, because if UGI is login - // using Kerberos keytabs, there is no HDFS delegation token in the UGI context. - String fileLocation = System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); - if (fileLocation != null) { + try (DataOutputBuffer dob = new DataOutputBuffer()) { Method readTokenStorageFileMethod = Credentials.class.getMethod( "readTokenStorageFile", File.class, org.apache.hadoop.conf.Configuration.class); @@ -509,10 +510,11 @@ static ContainerLaunchContext createTaskExecutorContext( cred.writeTokenStorageToStream(dob); ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); ctx.setTokens(securityTokens); + } catch (Throwable t) { + log.error("Failed to add Hadoop's security tokens.", t); } - } - catch (Throwable t) { - log.error("Getting current user info failed when trying to launch the container", t); + } else { + log.info("Could not set security tokens because Hadoop's token file location is unknown."); } return ctx; From f3389638eed8d48db013dadf72c1af9748491968 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 14 Mar 2018 14:21:27 +0100 Subject: [PATCH 161/268] [FLINK-8942][runtime] Pass heartbeat target ResourceID received payload field now volatile Add HeartbeatMonitor#getHeartbeatTargetId This closes #5699. --- .../runtime/heartbeat/HeartbeatListener.java | 3 +- .../heartbeat/HeartbeatManagerImpl.java | 8 +- .../heartbeat/HeartbeatManagerSenderImpl.java | 2 +- .../flink/runtime/jobmaster/JobMaster.java | 4 +- .../resourcemanager/ResourceManager.java | 4 +- .../runtime/taskexecutor/TaskExecutor.java | 4 +- .../heartbeat/HeartbeatManagerTest.java | 167 +++++++++++++++++- 7 files changed, 177 insertions(+), 15 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java index 734eb4c853b41..01a4754dfb116 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatListener.java @@ -57,7 +57,8 @@ public interface HeartbeatListener { * Retrieves the payload value for the next heartbeat message. Since the operation can happen * asynchronously, the result is returned wrapped in a future. * + * @param resourceID Resource ID identifying the receiver of the payload * @return Future containing the next payload for heartbeats */ - CompletableFuture retrievePayload(); + CompletableFuture retrievePayload(ResourceID resourceID); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java index 09c4b461b3f38..42268fc46e76d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerImpl.java @@ -106,7 +106,7 @@ HeartbeatListener getHeartbeatListener() { return heartbeatListener; } - Collection> getHeartbeatTargets() { + Collection> getHeartbeatTargets() { return heartbeatTargets.values(); } @@ -202,7 +202,7 @@ public void requestHeartbeat(final ResourceID requestOrigin, I heartbeatPayload) heartbeatListener.reportPayload(requestOrigin, heartbeatPayload); } - CompletableFuture futurePayload = heartbeatListener.retrievePayload(); + CompletableFuture futurePayload = heartbeatListener.retrievePayload(requestOrigin); if (futurePayload != null) { CompletableFuture sendHeartbeatFuture = futurePayload.thenAcceptAsync( @@ -289,6 +289,10 @@ HeartbeatTarget getHeartbeatTarget() { return heartbeatTarget; } + ResourceID getHeartbeatTargetId() { + return resourceID; + } + public long getLastHeartbeat() { return lastHeartbeat; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerSenderImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerSenderImpl.java index eb8234369b9d4..e3b939c068b72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerSenderImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerSenderImpl.java @@ -63,7 +63,7 @@ public void run() { if (!stopped) { log.debug("Trigger heartbeat request."); for (HeartbeatMonitor heartbeatMonitor : getHeartbeatTargets()) { - CompletableFuture futurePayload = getHeartbeatListener().retrievePayload(); + CompletableFuture futurePayload = getHeartbeatListener().retrievePayload(heartbeatMonitor.getHeartbeatTargetId()); final HeartbeatTarget heartbeatTarget = heartbeatMonitor.getHeartbeatTarget(); if (futurePayload != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index ced8c7c4dd89e..f0b29bf49a347 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -1527,7 +1527,7 @@ public void reportPayload(ResourceID resourceID, Void payload) { } @Override - public CompletableFuture retrievePayload() { + public CompletableFuture retrievePayload(ResourceID resourceID) { return CompletableFuture.completedFuture(null); } } @@ -1551,7 +1551,7 @@ public void reportPayload(ResourceID resourceID, Void payload) { } @Override - public CompletableFuture retrievePayload() { + public CompletableFuture retrievePayload(ResourceID resourceID) { return CompletableFuture.completedFuture(null); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 77e43621dbb20..0ae4ab6af3a08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -1076,7 +1076,7 @@ public void run() { } @Override - public CompletableFuture retrievePayload() { + public CompletableFuture retrievePayload(ResourceID resourceID) { return CompletableFuture.completedFuture(null); } } @@ -1109,7 +1109,7 @@ public void reportPayload(ResourceID resourceID, Void payload) { } @Override - public CompletableFuture retrievePayload() { + public CompletableFuture retrievePayload(ResourceID resourceID) { return CompletableFuture.completedFuture(null); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index fc69984c8a0c8..7409175d44aa1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -1515,7 +1515,7 @@ public void reportPayload(ResourceID resourceID, Void payload) { } @Override - public CompletableFuture retrievePayload() { + public CompletableFuture retrievePayload(ResourceID resourceID) { return CompletableFuture.completedFuture(null); } } @@ -1544,7 +1544,7 @@ public void reportPayload(ResourceID resourceID, Void payload) { } @Override - public CompletableFuture retrievePayload() { + public CompletableFuture retrievePayload(ResourceID resourceID) { return callAsync( () -> taskSlotTable.createSlotReport(getResourceID()), taskManagerConfiguration.getTimeout()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java index 390a1312e5413..77d12d54f0c6e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.heartbeat; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.ScheduledExecutor; @@ -75,7 +76,7 @@ public void testRegularHeartbeat() { Object expectedObject = new Object(); - when(heartbeatListener.retrievePayload()).thenReturn(CompletableFuture.completedFuture(expectedObject)); + when(heartbeatListener.retrievePayload(any(ResourceID.class))).thenReturn(CompletableFuture.completedFuture(expectedObject)); HeartbeatManagerImpl heartbeatManager = new HeartbeatManagerImpl<>( heartbeatTimeout, @@ -93,7 +94,7 @@ public void testRegularHeartbeat() { heartbeatManager.requestHeartbeat(targetResourceID, expectedObject); verify(heartbeatListener, times(1)).reportPayload(targetResourceID, expectedObject); - verify(heartbeatListener, times(1)).retrievePayload(); + verify(heartbeatListener, times(1)).retrievePayload(any(ResourceID.class)); verify(heartbeatTarget, times(1)).receiveHeartbeat(ownResourceID, expectedObject); heartbeatManager.receiveHeartbeat(targetResourceID, expectedObject); @@ -118,7 +119,7 @@ public void testHeartbeatMonitorUpdate() { Object expectedObject = new Object(); - when(heartbeatListener.retrievePayload()).thenReturn(CompletableFuture.completedFuture(expectedObject)); + when(heartbeatListener.retrievePayload(any(ResourceID.class))).thenReturn(CompletableFuture.completedFuture(expectedObject)); HeartbeatManagerImpl heartbeatManager = new HeartbeatManagerImpl<>( heartbeatTimeout, @@ -207,7 +208,7 @@ public void testHeartbeatCluster() throws Exception { @SuppressWarnings("unchecked") HeartbeatListener heartbeatListener = mock(HeartbeatListener.class); - when(heartbeatListener.retrievePayload()).thenReturn(CompletableFuture.completedFuture(object)); + when(heartbeatListener.retrievePayload(any(ResourceID.class))).thenReturn(CompletableFuture.completedFuture(object)); TestingHeartbeatListener heartbeatListener2 = new TestingHeartbeatListener(object2); @@ -347,6 +348,162 @@ public void testLastHeartbeatFrom() { } } + /** + * Tests that the heartbeat target {@link ResourceID} is properly passed to the {@link HeartbeatListener} by the + * {@link HeartbeatManagerImpl}. + */ + @Test + public void testHeartbeatManagerTargetPayload() { + final long heartbeatTimeout = 100L; + + final ResourceID someTargetId = ResourceID.generate(); + final ResourceID specialTargetId = ResourceID.generate(); + final TargetDependentHeartbeatReceiver someHeartbeatTarget = new TargetDependentHeartbeatReceiver(); + final TargetDependentHeartbeatReceiver specialHeartbeatTarget = new TargetDependentHeartbeatReceiver(); + + final int defaultResponse = 0; + final int specialResponse = 1; + + HeartbeatManager heartbeatManager = new HeartbeatManagerImpl<>( + heartbeatTimeout, + ResourceID.generate(), + new TargetDependentHeartbeatSender(specialTargetId, specialResponse, defaultResponse), + Executors.directExecutor(), + mock(ScheduledExecutor.class), + LOG); + + try { + heartbeatManager.monitorTarget(someTargetId, someHeartbeatTarget); + heartbeatManager.monitorTarget(specialTargetId, specialHeartbeatTarget); + + heartbeatManager.requestHeartbeat(someTargetId, null); + assertEquals(defaultResponse, someHeartbeatTarget.getLastReceivedHeartbeatPayload()); + + heartbeatManager.requestHeartbeat(specialTargetId, null); + assertEquals(specialResponse, specialHeartbeatTarget.getLastReceivedHeartbeatPayload()); + } finally { + heartbeatManager.stop(); + } + } + + /** + * Tests that the heartbeat target {@link ResourceID} is properly passed to the {@link HeartbeatListener} by the + * {@link HeartbeatManagerSenderImpl}. + */ + @Test + public void testHeartbeatManagerSenderTargetPayload() throws Exception { + final long heartbeatTimeout = 100L; + final long heartbeatPeriod = 2000L; + + final ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = new ScheduledThreadPoolExecutor(1); + + final ResourceID someTargetId = ResourceID.generate(); + final ResourceID specialTargetId = ResourceID.generate(); + + final OneShotLatch someTargetReceivedLatch = new OneShotLatch(); + final OneShotLatch specialTargetReceivedLatch = new OneShotLatch(); + + final TargetDependentHeartbeatReceiver someHeartbeatTarget = new TargetDependentHeartbeatReceiver(someTargetReceivedLatch); + final TargetDependentHeartbeatReceiver specialHeartbeatTarget = new TargetDependentHeartbeatReceiver(specialTargetReceivedLatch); + + final int defaultResponse = 0; + final int specialResponse = 1; + + HeartbeatManager heartbeatManager = new HeartbeatManagerSenderImpl<>( + heartbeatPeriod, + heartbeatTimeout, + ResourceID.generate(), + new TargetDependentHeartbeatSender(specialTargetId, specialResponse, defaultResponse), + Executors.directExecutor(), + new ScheduledExecutorServiceAdapter(scheduledThreadPoolExecutor), + LOG); + + try { + heartbeatManager.monitorTarget(someTargetId, someHeartbeatTarget); + heartbeatManager.monitorTarget(specialTargetId, specialHeartbeatTarget); + + someTargetReceivedLatch.await(5, TimeUnit.SECONDS); + specialTargetReceivedLatch.await(5, TimeUnit.SECONDS); + + assertEquals(defaultResponse, someHeartbeatTarget.getLastRequestedHeartbeatPayload()); + assertEquals(specialResponse, specialHeartbeatTarget.getLastRequestedHeartbeatPayload()); + } finally { + heartbeatManager.stop(); + scheduledThreadPoolExecutor.shutdown(); + } + } + + /** + * Test {@link HeartbeatTarget} that exposes the last received payload. + */ + private static class TargetDependentHeartbeatReceiver implements HeartbeatTarget { + + private volatile int lastReceivedHeartbeatPayload = -1; + private volatile int lastRequestedHeartbeatPayload = -1; + + private final OneShotLatch latch; + + public TargetDependentHeartbeatReceiver() { + this(new OneShotLatch()); + } + + public TargetDependentHeartbeatReceiver(OneShotLatch latch) { + this.latch = latch; + } + + @Override + public void receiveHeartbeat(ResourceID heartbeatOrigin, Integer heartbeatPayload) { + this.lastReceivedHeartbeatPayload = heartbeatPayload; + latch.trigger(); + } + + @Override + public void requestHeartbeat(ResourceID requestOrigin, Integer heartbeatPayload) { + this.lastRequestedHeartbeatPayload = heartbeatPayload; + latch.trigger(); + } + + public int getLastReceivedHeartbeatPayload() { + return lastReceivedHeartbeatPayload; + } + + public int getLastRequestedHeartbeatPayload() { + return lastRequestedHeartbeatPayload; + } + } + + /** + * Test {@link HeartbeatListener} that returns different payloads based on the target {@link ResourceID}. + */ + private static class TargetDependentHeartbeatSender implements HeartbeatListener { + private final ResourceID specialId; + private final int specialResponse; + private final int defaultResponse; + + TargetDependentHeartbeatSender(ResourceID specialId, int specialResponse, int defaultResponse) { + this.specialId = specialId; + this.specialResponse = specialResponse; + this.defaultResponse = defaultResponse; + } + + @Override + public void notifyHeartbeatTimeout(ResourceID resourceID) { + } + + @Override + public void reportPayload(ResourceID resourceID, Object payload) { + } + + @Override + public CompletableFuture retrievePayload(ResourceID resourceID) { + if (resourceID.equals(specialId)) { + return CompletableFuture.completedFuture(specialResponse); + } else { + return CompletableFuture.completedFuture(defaultResponse); + } + } + } + static class TestingHeartbeatListener implements HeartbeatListener { private final CompletableFuture future = new CompletableFuture<>(); @@ -378,7 +535,7 @@ public void reportPayload(ResourceID resourceID, Object payload) { } @Override - public CompletableFuture retrievePayload() { + public CompletableFuture retrievePayload(ResourceID resourceID) { return CompletableFuture.completedFuture(payload); } } From e2a62b3e257a0625392209be0a93880988c0c6f7 Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 20 Mar 2018 10:02:21 +0800 Subject: [PATCH 162/268] [FLINK-9019] Unclosed closeableRegistry in StreamTaskStateInitializerImpl#rawOperatorStateInputs This closes #5723. --- .../api/operators/StreamTaskStateInitializerImpl.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index acbc2f8bc9ed5..7e915544e4c11 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -279,8 +279,6 @@ protected CloseableIterable rawOperatorStateInputs if (restoreStateAlternatives.hasNext()) { - final CloseableRegistry closeableRegistry = new CloseableRegistry(); - Collection rawOperatorState = restoreStateAlternatives.next(); // TODO currently this does not support local state recovery, so we expect there is only one handle. Preconditions.checkState( @@ -288,8 +286,10 @@ protected CloseableIterable rawOperatorStateInputs "Local recovery is currently not implemented for raw operator state, but found state alternative."); if (rawOperatorState != null) { - return new CloseableIterable() { + + final CloseableRegistry closeableRegistry = new CloseableRegistry(); + @Override public void close() throws IOException { closeableRegistry.close(); From 071dedcb769d1e3e899134e9566c69808ba99c53 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 27 Feb 2018 17:23:20 +0100 Subject: [PATCH 163/268] [FLINK-8801][yarn/s3] fix Utils#setupLocalResource() relying on consistent read-after-write "Amazon S3 provides read-after-write consistency for PUTS of new objects in your S3 bucket in all regions with one caveat. The caveat is that if you make a HEAD or GET request to the key name (to find if the object exists) before creating the object, Amazon S3 provides eventual consistency for read-after-write." https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel Some S3 file system implementations may actually execute such a request for the about-to-write object and thus the read-after-write is only eventually consistent. org.apache.flink.yarn.Utils#setupLocalResource() currently relies on a consistent read-after-write since it accesses the remote resource to get file size and modification timestamp. Since there we have access to the local resource, we can use this metadata directly instead and circumvent the problem. This closes #5602. --- .../java/org/apache/flink/yarn/Utils.java | 34 +++++++++++++++++-- 1 file changed, 32 insertions(+), 2 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java index b9f7fac297862..b895784766f47 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java @@ -141,7 +141,8 @@ static Tuple2 setupLocalResource( Path homedir, String relativeTargetPath) throws IOException { - if (new File(localSrcPath.toUri().getPath()).isDirectory()) { + File localFile = new File(localSrcPath.toUri().getPath()); + if (localFile.isDirectory()) { throw new IllegalArgumentException("File to copy must not be a directory: " + localSrcPath); } @@ -159,11 +160,40 @@ static Tuple2 setupLocalResource( fs.copyFromLocalFile(false, true, localSrcPath, dst); + // Note: If we used registerLocalResource(FileSystem, Path) here, we would access the remote + // file once again which has problems with eventually consistent read-after-write file + // systems. Instead, we decide to preserve the modification time at the remote + // location because this and the size of the resource will be checked by YARN based on + // the values we provide to #registerLocalResource() below. + fs.setTimes(dst, localFile.lastModified(), -1); // now create the resource instance - LocalResource resource = registerLocalResource(fs, dst); + LocalResource resource = registerLocalResource(dst, localFile.length(), localFile.lastModified()); + return Tuple2.of(dst, resource); } + /** + * Creates a YARN resource for the remote object at the given location. + * + * @param remoteRsrcPath remote location of the resource + * @param resourceSize size of the resource + * @param resourceModificationTime last modification time of the resource + * + * @return YARN resource + */ + private static LocalResource registerLocalResource( + Path remoteRsrcPath, + long resourceSize, + long resourceModificationTime) { + LocalResource localResource = Records.newRecord(LocalResource.class); + localResource.setResource(ConverterUtils.getYarnUrlFromURI(remoteRsrcPath.toUri())); + localResource.setSize(resourceSize); + localResource.setTimestamp(resourceModificationTime); + localResource.setType(LocalResourceType.FILE); + localResource.setVisibility(LocalResourceVisibility.APPLICATION); + return localResource; + } + private static LocalResource registerLocalResource(FileSystem fs, Path remoteRsrcPath) throws IOException { LocalResource localResource = Records.newRecord(LocalResource.class); FileStatus jarStat = fs.getFileStatus(remoteRsrcPath); From 6657aa2090b6b0f56f329152b3aaa1c147e73380 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 14 Mar 2018 18:52:16 +0100 Subject: [PATCH 164/268] [FLINK-8881][runtime] Send accumulator updates via heartbeats --- .../flink/runtime/jobmaster/JobMaster.java | 16 +++++--- .../runtime/jobmaster/JobMasterGateway.java | 6 ++- .../taskexecutor/AccumulatorReport.java | 40 +++++++++++++++++++ .../runtime/taskexecutor/TaskExecutor.java | 32 +++++++++++---- .../utils/TestingJobMasterGateway.java | 3 +- 5 files changed, 81 insertions(+), 16 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/AccumulatorReport.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index f0b29bf49a347..6878032f0e17f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -27,6 +27,7 @@ import org.apache.flink.queryablestate.KvStateID; import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.StoppingException; +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; import org.apache.flink.runtime.checkpoint.CheckpointDeclineReason; @@ -93,6 +94,7 @@ import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.taskexecutor.AccumulatorReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; @@ -166,7 +168,7 @@ public class JobMaster extends FencedRpcEndpoint implements JobMast private final JobManagerJobMetricGroup jobMetricGroup; /** The heartbeat manager with task managers. */ - private final HeartbeatManager taskManagerHeartbeatManager; + private final HeartbeatManager taskManagerHeartbeatManager; /** The heartbeat manager with resource manager. */ private final HeartbeatManager resourceManagerHeartbeatManager; @@ -938,8 +940,8 @@ public void disconnectResourceManager( } @Override - public void heartbeatFromTaskManager(final ResourceID resourceID) { - taskManagerHeartbeatManager.receiveHeartbeat(resourceID, null); + public void heartbeatFromTaskManager(final ResourceID resourceID, AccumulatorReport accumulatorReport) { + taskManagerHeartbeatManager.receiveHeartbeat(resourceID, accumulatorReport); } @Override @@ -1504,7 +1506,7 @@ public void jobStatusChanges( } } - private class TaskManagerHeartbeatListener implements HeartbeatListener { + private class TaskManagerHeartbeatListener implements HeartbeatListener { private final JobMasterGateway jobMasterGateway; @@ -1522,8 +1524,10 @@ public void notifyHeartbeatTimeout(ResourceID resourceID) { } @Override - public void reportPayload(ResourceID resourceID, Void payload) { - // nothing to do since there is no payload + public void reportPayload(ResourceID resourceID, AccumulatorReport payload) { + for (AccumulatorSnapshot snapshot : payload.getAccumulatorSnapshots()) { + executionGraph.updateAccumulators(snapshot); + } } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java index 1e1bdda45117a..4ea93577fe3a1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse; import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; +import org.apache.flink.runtime.taskexecutor.AccumulatorReport; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; @@ -219,8 +220,11 @@ CompletableFuture registerTaskManager( * Sends the heartbeat to job manager from task manager. * * @param resourceID unique id of the task manager + * @param accumulatorReport report containing accumulator updates */ - void heartbeatFromTaskManager(final ResourceID resourceID); + void heartbeatFromTaskManager( + final ResourceID resourceID, + final AccumulatorReport accumulatorReport); /** * Sends heartbeat request from the resource manager. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/AccumulatorReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/AccumulatorReport.java new file mode 100644 index 0000000000000..7c8c767fe0c99 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/AccumulatorReport.java @@ -0,0 +1,40 @@ +/* + * 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.taskexecutor; + +import org.apache.flink.runtime.accumulators.AccumulatorSnapshot; + +import java.io.Serializable; +import java.util.Collection; +import java.util.List; + +/** + * A report about the current values of all accumulators of the TaskExecutor for a given job. + */ +public class AccumulatorReport implements Serializable { + private final Collection accumulatorSnapshots; + + public AccumulatorReport(List accumulatorSnapshots) { + this.accumulatorSnapshots = accumulatorSnapshots; + } + + public Collection getAccumulatorSnapshots() { + return accumulatorSnapshots; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index 7409175d44aa1..f25601e534b0c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -107,6 +107,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -138,7 +139,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final TaskManagerConfiguration taskManagerConfiguration; /** The heartbeat manager for job manager in the task manager. */ - private final HeartbeatManager jobManagerHeartbeatManager; + private final HeartbeatManager jobManagerHeartbeatManager; /** The heartbeat manager for resource manager in the task manager. */ private final HeartbeatManager resourceManagerHeartbeatManager; @@ -1050,14 +1051,14 @@ private void establishJobManagerConnection(JobID jobId, final JobMasterGateway j jobManagerTable.put(jobId, newJobManagerConnection); // monitor the job manager as heartbeat target - jobManagerHeartbeatManager.monitorTarget(jobManagerResourceID, new HeartbeatTarget() { + jobManagerHeartbeatManager.monitorTarget(jobManagerResourceID, new HeartbeatTarget() { @Override - public void receiveHeartbeat(ResourceID resourceID, Void payload) { - jobMasterGateway.heartbeatFromTaskManager(resourceID); + public void receiveHeartbeat(ResourceID resourceID, AccumulatorReport payload) { + jobMasterGateway.heartbeatFromTaskManager(resourceID, payload); } @Override - public void requestHeartbeat(ResourceID resourceID, Void payload) { + public void requestHeartbeat(ResourceID resourceID, AccumulatorReport payload) { // request heartbeat will never be called on the task manager side } }); @@ -1488,7 +1489,7 @@ public void timeoutSlot(final AllocationID allocationId, final UUID ticket) { } } - private class JobManagerHeartbeatListener implements HeartbeatListener { + private class JobManagerHeartbeatListener implements HeartbeatListener { @Override public void notifyHeartbeatTimeout(final ResourceID resourceID) { @@ -1515,8 +1516,23 @@ public void reportPayload(ResourceID resourceID, Void payload) { } @Override - public CompletableFuture retrievePayload(ResourceID resourceID) { - return CompletableFuture.completedFuture(null); + public CompletableFuture retrievePayload(ResourceID resourceID) { + validateRunsInMainThread(); + JobManagerConnection jobManagerConnection = jobManagerConnections.get(resourceID); + if (jobManagerConnection != null) { + JobID jobId = jobManagerConnection.getJobID(); + + List accumulatorSnapshots = new ArrayList<>(16); + Iterator allTasks = taskSlotTable.getTasks(jobId); + + while (allTasks.hasNext()) { + Task task = allTasks.next(); + accumulatorSnapshots.add(task.getAccumulatorRegistry().getSnapshot()); + } + return CompletableFuture.completedFuture(new AccumulatorReport(accumulatorSnapshots)); + } else { + return CompletableFuture.completedFuture(new AccumulatorReport(Collections.emptyList())); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java index 0d57a56b2ceb3..65117af970b7d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/TestingJobMasterGateway.java @@ -44,6 +44,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rest.handler.legacy.backpressure.OperatorBackPressureStatsResponse; import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.taskexecutor.AccumulatorReport; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; @@ -130,7 +131,7 @@ public CompletableFuture registerTaskManager(String taskMa } @Override - public void heartbeatFromTaskManager(ResourceID resourceID) { + public void heartbeatFromTaskManager(ResourceID resourceID, AccumulatorReport accumulatorReport) { throw new UnsupportedOperationException(); } From 60e05c05f3fbf20c1276bc2f1006eb4224eda43f Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 26 Feb 2018 14:54:07 +0100 Subject: [PATCH 165/268] [FLINK-8703][tests] Port SavepointMigrationTestBase to MiniClusterResource This closes #5701. --- ...cyStatefulJobSavepointMigrationITCase.java | 2 +- .../utils/SavepointMigrationTestBase.java | 99 +++++++------------ .../StatefulJobSavepointMigrationITCase.java | 2 +- 3 files changed, 39 insertions(+), 64 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/LegacyStatefulJobSavepointMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/LegacyStatefulJobSavepointMigrationITCase.java index 45a691163e154..eee13500e3c60 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/LegacyStatefulJobSavepointMigrationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/LegacyStatefulJobSavepointMigrationITCase.java @@ -89,7 +89,7 @@ public static Collection> parameters () { private final MigrationVersion testMigrateVersion; private final String testStateBackend; - public LegacyStatefulJobSavepointMigrationITCase(Tuple2 testMigrateVersionAndBackend) { + public LegacyStatefulJobSavepointMigrationITCase(Tuple2 testMigrateVersionAndBackend) throws Exception { this.testMigrateVersion = testMigrateVersionAndBackend.f0; this.testStateBackend = testMigrateVersionAndBackend.f1; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java index 28825040d4811..91b5de8ca5e25 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java @@ -21,25 +21,21 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.client.program.StandaloneClusterClient; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.savepoint.SavepointSerializers; -import org.apache.flink.runtime.client.JobListeningContext; -import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.test.util.TestBaseUtils; import org.apache.commons.io.FileUtils; -import org.junit.After; -import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; @@ -49,34 +45,35 @@ import java.net.URI; import java.net.URL; import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import scala.Option; -import scala.concurrent.Await; -import scala.concurrent.Future; import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; import static junit.framework.Assert.fail; +import static org.junit.Assert.assertNotEquals; /** * Test savepoint migration. */ -public class SavepointMigrationTestBase extends TestBaseUtils { +public abstract class SavepointMigrationTestBase extends TestBaseUtils { @BeforeClass public static void before() { SavepointSerializers.setFailWhenLegacyStateDetected(false); } + @ClassRule + public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + @Rule - public TemporaryFolder tempFolder = new TemporaryFolder(); + public final MiniClusterResource miniClusterResource; private static final Logger LOG = LoggerFactory.getLogger(SavepointMigrationTestBase.class); private static final Deadline DEADLINE = new FiniteDuration(5, TimeUnit.MINUTES).fromNow(); protected static final int DEFAULT_PARALLELISM = 4; - protected LocalFlinkMiniCluster cluster = null; protected static String getResourceFilename(String filename) { ClassLoader cl = SavepointMigrationTestBase.class.getClassLoader(); @@ -87,17 +84,25 @@ protected static String getResourceFilename(String filename) { return resource.getFile(); } - @Before - public void setup() throws Exception { + protected SavepointMigrationTestBase() throws Exception { + miniClusterResource = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfiguration(), + 1, + DEFAULT_PARALLELISM), + true); + } + private Configuration getConfiguration() throws Exception { // Flink configuration final Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, DEFAULT_PARALLELISM); - final File checkpointDir = tempFolder.newFolder("checkpoints").getAbsoluteFile(); - final File savepointDir = tempFolder.newFolder("savepoints").getAbsoluteFile(); + UUID id = UUID.randomUUID(); + final File checkpointDir = TEMP_FOLDER.newFolder("checkpoints_" + id).getAbsoluteFile(); + final File savepointDir = TEMP_FOLDER.newFolder("savepoints_" + id).getAbsoluteFile(); if (!checkpointDir.exists() || !savepointDir.exists()) { throw new Exception("Test setup failed: failed to create (temporary) directories."); @@ -111,12 +116,7 @@ public void setup() throws Exception { config.setInteger(CheckpointingOptions.FS_SMALL_FILE_THRESHOLD, 0); config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString()); - cluster = TestBaseUtils.startCluster(config, false); - } - - @After - public void teardown() throws Exception { - stopCluster(cluster, TestBaseUtils.DEFAULT_TIMEOUT); + return config; } @SafeVarargs @@ -125,22 +125,20 @@ protected final void executeAndSavepoint( String savepointPath, Tuple2... expectedAccumulators) throws Exception { - // Retrieve the job manager - ActorGateway jobManager = Await.result(cluster.leaderGateway().future(), DEADLINE.timeLeft()); + ClusterClient client = miniClusterResource.getClusterClient(); + client.setDetached(true); // Submit the job JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - JobSubmissionResult jobSubmissionResult = cluster.submitJobDetached(jobGraph); + JobSubmissionResult jobSubmissionResult = client.submitJob(jobGraph, SavepointMigrationTestBase.class.getClassLoader()); LOG.info("Submitted job {} and waiting...", jobSubmissionResult.getJobID()); - StandaloneClusterClient clusterClient = new StandaloneClusterClient(cluster.configuration()); - boolean done = false; while (DEADLINE.hasTimeLeft()) { Thread.sleep(100); - Map accumulators = clusterClient.getAccumulators(jobSubmissionResult.getJobID()); + Map accumulators = client.getAccumulators(jobSubmissionResult.getJobID()); boolean allDone = true; for (Tuple2 acc : expectedAccumulators) { @@ -166,18 +164,9 @@ protected final void executeAndSavepoint( LOG.info("Triggering savepoint."); - final Future savepointResultFuture = - jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobSubmissionResult.getJobID(), Option.empty()), DEADLINE.timeLeft()); + CompletableFuture savepointPathFuture = client.triggerSavepoint(jobSubmissionResult.getJobID(), null); - Object savepointResult = Await.result(savepointResultFuture, DEADLINE.timeLeft()); - - if (savepointResult instanceof JobManagerMessages.TriggerSavepointFailure) { - fail("Error drawing savepoint: " + ((JobManagerMessages.TriggerSavepointFailure) savepointResult).cause()); - } - - // jobmanager will store savepoint in heap, we have to retrieve it - final String jobmanagerSavepointPath = ((JobManagerMessages.TriggerSavepointSuccess) savepointResult).savepointPath(); - LOG.info("Saved savepoint: " + jobmanagerSavepointPath); + String jobmanagerSavepointPath = savepointPathFuture.get(DEADLINE.timeLeft().toMillis(), TimeUnit.MILLISECONDS); File jobManagerSavepoint = new File(new URI(jobmanagerSavepointPath).getPath()); // savepoints were changed to be directories in Flink 1.3 @@ -194,18 +183,15 @@ protected final void restoreAndExecute( String savepointPath, Tuple2... expectedAccumulators) throws Exception { - // Retrieve the job manager - Await.result(cluster.leaderGateway().future(), DEADLINE.timeLeft()); + ClusterClient client = miniClusterResource.getClusterClient(); + client.setDetached(true); // Submit the job JobGraph jobGraph = env.getStreamGraph().getJobGraph(); jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - JobSubmissionResult jobSubmissionResult = cluster.submitJobDetached(jobGraph); - - StandaloneClusterClient clusterClient = new StandaloneClusterClient(cluster.configuration()); - JobListeningContext jobListeningContext = clusterClient.connectToJob(jobSubmissionResult.getJobID()); + JobSubmissionResult jobSubmissionResult = client.submitJob(jobGraph, SavepointMigrationTestBase.class.getClassLoader()); boolean done = false; while (DEADLINE.hasTimeLeft()) { @@ -213,30 +199,19 @@ protected final void restoreAndExecute( // try and get a job result, this will fail if the job already failed. Use this // to get out of this loop JobID jobId = jobSubmissionResult.getJobID(); - FiniteDuration timeout = FiniteDuration.apply(5, TimeUnit.SECONDS); try { + CompletableFuture jobStatusFuture = client.getJobStatus(jobSubmissionResult.getJobID()); - Future future = clusterClient - .getJobManagerGateway() - .ask(JobManagerMessages.getRequestJobStatus(jobSubmissionResult.getJobID()), timeout); + JobStatus jobStatus = jobStatusFuture.get(5, TimeUnit.SECONDS); - Object result = Await.result(future, timeout); - - if (result instanceof JobManagerMessages.CurrentJobStatus) { - if (((JobManagerMessages.CurrentJobStatus) result).status() == JobStatus.FAILED) { - Object jobResult = Await.result( - jobListeningContext.getJobResultFuture(), - Duration.apply(5, TimeUnit.SECONDS)); - fail("Job failed: " + jobResult); - } - } + assertNotEquals(JobStatus.FAILED, jobStatus); } catch (Exception e) { fail("Could not connect to job: " + e); } Thread.sleep(100); - Map accumulators = clusterClient.getAccumulators(jobId); + Map accumulators = client.getAccumulators(jobId); boolean allDone = true; for (Tuple2 acc : expectedAccumulators) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java index 53a535323861d..d2de8810eb6d4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/StatefulJobSavepointMigrationITCase.java @@ -95,7 +95,7 @@ public static Collection> parameters () { private final MigrationVersion testMigrateVersion; private final String testStateBackend; - public StatefulJobSavepointMigrationITCase(Tuple2 testMigrateVersionAndBackend) { + public StatefulJobSavepointMigrationITCase(Tuple2 testMigrateVersionAndBackend) throws Exception { this.testMigrateVersion = testMigrateVersionAndBackend.f0; this.testStateBackend = testMigrateVersionAndBackend.f1; } From bfd8229d8fccd9a06266692372b3ed35ae5d07d1 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Mar 2018 15:43:33 +0100 Subject: [PATCH 166/268] [hotfix] [core] Add @FunctionalInterface to KeySelector That clarifies that this interface should always be a SAM interface to allow that users created lambdas for its use. --- .../flink/api/java/functions/KeySelector.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java b/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java index 4aa84693318c2..5594f9ebf3b09 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/functions/KeySelector.java @@ -27,19 +27,20 @@ * The {@link KeySelector} allows to use deterministic objects for operations such as * reduce, reduceGroup, join, coGroup, etc. If invoked multiple times on the same object, * the returned key must be the same. - * - * The extractor takes an object and returns the deterministic key for that object. + * + *

    The extractor takes an object and returns the deterministic key for that object. * * @param Type of objects to extract the key from. * @param Type of key. */ @Public +@FunctionalInterface public interface KeySelector extends Function, Serializable { /** * User-defined function that deterministically extracts the key from an object. - * - * For example for a class: + * + *

    For example for a class: *

     	 * 	public class Word {
     	 * 		String word;
    @@ -48,19 +49,19 @@ public interface KeySelector extends Function, Serializable {
     	 * 
    * The key extractor could return the word as * a key to group all Word objects by the String they contain. - * - * The code would look like this + * + *

    The code would look like this *

     	 * 	public String getKey(Word w) {
     	 * 		return w.word;
     	 * 	}
     	 * 
    - * + * * @param value The object to get the key from. * @return The extracted key. - * + * * @throws Exception Throwing an exception will cause the execution of the respective task to fail, - * and trigger recovery or cancellation of the program. + * and trigger recovery or cancellation of the program. */ KEY getKey(IN value) throws Exception; } From fecc19088b36fc4c8bca5ff39ba756f8fd111171 Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Tue, 20 Mar 2018 15:59:33 +0800 Subject: [PATCH 167/268] [FLINK-9028] [yarn] Perform parameters checking before Yarn starting cluster This closes #5726. --- .../ContaineredTaskManagerParameters.java | 41 +++++++++++++------ .../flink/runtime/dispatcher/Dispatcher.java | 2 +- .../ContaineredTaskManagerParametersTest.java | 32 +++++++++++++++ .../taskexecutor/TaskManagerServicesTest.java | 1 - .../yarn/AbstractYarnClusterDescriptor.java | 27 ++++++++++++ .../flink/yarn/cli/FlinkYarnSessionCli.java | 3 +- .../flink/yarn/FlinkYarnSessionCliTest.java | 4 +- .../flink/yarn/YarnClusterDescriptorTest.java | 15 ++++--- 8 files changed, 103 insertions(+), 22 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java index c4dd486c5aeb7..fa7fdf445a61c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.runtime.taskexecutor.TaskManagerServices; +import org.apache.flink.util.Preconditions; import java.util.HashMap; import java.util.Map; @@ -101,46 +102,62 @@ public String toString() { // ------------------------------------------------------------------------ // Factory // ------------------------------------------------------------------------ - + /** - * Computes the parameters to be used to start a TaskManager Java process. + * calcuate cutoff memory size used by container, it will throw an {@link IllegalArgumentException} + * if the config is invalid or return the cutoff value if valid. * * @param config The Flink configuration. * @param containerMemoryMB The size of the complete container, in megabytes. - * @return The parameters to start the TaskManager processes with. + * + * @return cutoff memory size used by container. */ - public static ContaineredTaskManagerParameters create( - Configuration config, long containerMemoryMB, int numSlots) - { - // (1) compute how much memory we subtract from the total memory, to get the Java memory + public static long calculateCutoffMB(Configuration config, long containerMemoryMB) { + Preconditions.checkArgument(containerMemoryMB > 0); + // (1) check cutoff ratio final float memoryCutoffRatio = config.getFloat( ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO); - final int minCutoff = config.getInteger( - ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN); - if (memoryCutoffRatio >= 1 || memoryCutoffRatio <= 0) { throw new IllegalArgumentException("The configuration value '" + ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO.key() + "' must be between 0 and 1. Value given=" + memoryCutoffRatio); } + // (2) check min cutoff value + final int minCutoff = config.getInteger( + ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN); + if (minCutoff >= containerMemoryMB) { throw new IllegalArgumentException("The configuration value '" + ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN.key() + "'='" + minCutoff + "' is larger than the total container memory " + containerMemoryMB); } + // (3) check between heap and off-heap long cutoff = (long) (containerMemoryMB * memoryCutoffRatio); if (cutoff < minCutoff) { cutoff = minCutoff; } + return cutoff; + } - final long javaMemorySizeMB = containerMemoryMB - cutoff; + /** + * Computes the parameters to be used to start a TaskManager Java process. + * + * @param config The Flink configuration. + * @param containerMemoryMB The size of the complete container, in megabytes. + * @return The parameters to start the TaskManager processes with. + */ + public static ContaineredTaskManagerParameters create( + Configuration config, long containerMemoryMB, int numSlots) + { + // (1) try to compute how much memory used by container + final long cutoffMB = calculateCutoffMB(config, containerMemoryMB); // (2) split the remaining Java memory between heap and off-heap - final long heapSizeMB = TaskManagerServices.calculateHeapSizeMB(javaMemorySizeMB, config); + final long heapSizeMB = TaskManagerServices.calculateHeapSizeMB(containerMemoryMB - cutoffMB, config); // use the cut-off memory for off-heap (that was its intention) final long offHeapSizeMB = containerMemoryMB - heapSizeMB; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 91a4f73bf5384..68b40468a29b8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -313,7 +313,7 @@ public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) @Override public CompletableFuture> listJobs(Time timeout) { if (jobManagerRunners.isEmpty()) { - System.out.println("empty"); + log.info("empty"); } return CompletableFuture.completedFuture( Collections.unmodifiableSet(new HashSet<>(jobManagerRunners.keySet()))); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java index 8d9ea88a57556..230a9340cf8f6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -27,6 +28,7 @@ import static org.apache.flink.runtime.taskexecutor.TaskManagerServices.calculateNetworkBufferMemory; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class ContaineredTaskManagerParametersTest extends TestLogger { private static final long CONTAINER_MEMORY = 8192L; @@ -91,4 +93,34 @@ public void testTotalMemoryDoesNotExceedContainerMemoryOffHeap() { assertTrue(params.taskManagerHeapSizeMB() + params.taskManagerDirectMemoryLimitMB() <= CONTAINER_MEMORY); } + + /** + * Test to guard {@link ContaineredTaskManagerParameters#calculateCutoffMB(Configuration, long)}. + */ + @Test + public void testCalculateCutoffMB() throws Exception { + + Configuration config = new Configuration(); + long containerMemoryMB = 1000; + + config.setFloat(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO, 0.1f); + config.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 128); + + assertEquals(128, + ContaineredTaskManagerParameters.calculateCutoffMB(config, containerMemoryMB)); + + config.setFloat(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO, 0.2f); + assertEquals(200, + ContaineredTaskManagerParameters.calculateCutoffMB(config, containerMemoryMB)); + + config.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 1000); + + try { + ContaineredTaskManagerParameters.calculateCutoffMB(config, containerMemoryMB); + } catch (IllegalArgumentException expected) { + // we expected it. + return; + } + fail(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java index b0c6c60a9b2b9..d3d5444b67c97 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java @@ -214,5 +214,4 @@ public void calculateHeapSizeMB() throws Exception { config.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, 0.1f); // 10% assertEquals(810, TaskManagerServices.calculateHeapSizeMB(1000, config)); } - } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index bdb471a142f96..eab5e39f2cffb 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -36,9 +36,11 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.taskexecutor.TaskManagerServices; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.ShutdownHookUtil; @@ -409,6 +411,28 @@ public void terminateCluster(ApplicationId applicationId) throws FlinkException } } + /** + * Method to validate cluster specification before deploy it, it will throw + * an {@link IllegalConfigurationException} if the {@link ClusterSpecification} is invalid. + */ + private void validateClusterSpecification(ClusterSpecification clusterSpecification) { + long taskManagerMemorySize = clusterSpecification.getTaskManagerMemoryMB(); + long cutoff; + try { + // We do the validation by calling the calculation methods here + cutoff = ContaineredTaskManagerParameters.calculateCutoffMB(flinkConfiguration, taskManagerMemorySize); + } catch (IllegalArgumentException cutoffConfigurationInvalidEx) { + throw new IllegalConfigurationException("Configurations related to cutoff checked failed.", cutoffConfigurationInvalidEx); + } + + try { + // We do the validation by calling the calculation methods here + TaskManagerServices.calculateHeapSizeMB(taskManagerMemorySize - cutoff, flinkConfiguration); + } catch (IllegalArgumentException heapSizeConfigurationInvalidEx) { + throw new IllegalConfigurationException("Configurations related to heap size checked failed.", heapSizeConfigurationInvalidEx); + } + } + /** * This method will block until the ApplicationMaster/JobManager have been deployed on YARN. * @@ -423,6 +447,9 @@ protected ClusterClient deployInternal( @Nullable JobGraph jobGraph, boolean detached) throws Exception { + // ------------------ Check if configuration is valid -------------------- + validateClusterSpecification(clusterSpecification); + if (UserGroupInformation.isSecurityEnabled()) { // note: UGI::hasKerberosCredentials inaccurately reports false // for logins based on a keytab (fixed in Hadoop 2.6.1, see HADOOP-10786), diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 2cdc19d3c9360..1443f9957cd35 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -31,6 +31,7 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.security.SecurityConfiguration; @@ -631,7 +632,7 @@ public int run(String[] args) throws CliArgsException, FlinkException { if (detachedMode) { LOG.info("The Flink YARN client has been started in detached mode. In order to stop " + "Flink on YARN, use the following command or a YARN web interface to stop it:\n" + - "yarn application -kill " + applicationId.getOpt()); + "yarn application -kill " + yarnApplicationId); } else { ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index 20ce314399f5b..5b0d42219bedc 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -264,8 +264,8 @@ public void testCommandLineClusterSpecification() throws Exception { configuration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, 7331); configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); - final int jobManagerMemory = 42; - final int taskManagerMemory = 41; + final int jobManagerMemory = 1337; + final int taskManagerMemory = 7331; final int slotsPerTaskManager = 30; final String[] args = {"-yjm", String.valueOf(jobManagerMemory), "-ytm", String.valueOf(taskManagerMemory), "-ys", String.valueOf(slotsPerTaskManager)}; final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java index dd8b62536c260..52bf8bb1a6a83 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnClusterDescriptorTest.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.util.TestLogger; import org.apache.flink.yarn.cli.FlinkYarnSessionCli; @@ -91,8 +92,11 @@ public static void tearDownClass() { @Test public void testFailIfTaskSlotsHigherThanMaxVcores() throws ClusterDeploymentException { + final Configuration flinkConfiguration = new Configuration(); + flinkConfiguration.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 0); + YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( - new Configuration(), + flinkConfiguration, yarnConfiguration, temporaryFolder.getRoot().getAbsolutePath(), yarnClient, @@ -101,8 +105,8 @@ public void testFailIfTaskSlotsHigherThanMaxVcores() throws ClusterDeploymentExc clusterDescriptor.setLocalJarPath(new Path(flinkJar.getPath())); ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(-1) - .setTaskManagerMemoryMB(-1) + .setMasterMemoryMB(1) + .setTaskManagerMemoryMB(1) .setNumberTaskManagers(1) .setSlotsPerTaskManager(Integer.MAX_VALUE) .createClusterSpecification(); @@ -126,6 +130,7 @@ public void testConfigOverwrite() throws ClusterDeploymentException { Configuration configuration = new Configuration(); // overwrite vcores in config configuration.setInteger(YarnConfigOptions.VCORES, Integer.MAX_VALUE); + configuration.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 0); YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( configuration, @@ -138,8 +143,8 @@ public void testConfigOverwrite() throws ClusterDeploymentException { // configure slots ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() - .setMasterMemoryMB(-1) - .setTaskManagerMemoryMB(-1) + .setMasterMemoryMB(1) + .setTaskManagerMemoryMB(1) .setNumberTaskManagers(1) .setSlotsPerTaskManager(1) .createClusterSpecification(); From c6f91334b67d589f0c17ed75c9dbcbaedaf8ba51 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 20 Mar 2018 15:38:02 +0100 Subject: [PATCH 168/268] [FLINK-9028] [yarn] Improve failure message if cluster cannot be started --- .../ContaineredTaskManagerParameters.java | 28 +++++++++---------- .../ContaineredTaskManagerParametersTest.java | 9 +++--- .../taskexecutor/TaskManagerServicesTest.java | 2 +- .../yarn/AbstractYarnClusterDescriptor.java | 26 ++++++++--------- .../flink/yarn/cli/FlinkYarnSessionCli.java | 1 - .../flink/yarn/FlinkYarnSessionCliTest.java | 9 +++--- 6 files changed, 37 insertions(+), 38 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java index fa7fdf445a61c..a4e7d25017ba4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParameters.java @@ -32,23 +32,22 @@ public class ContaineredTaskManagerParameters implements java.io.Serializable { private static final long serialVersionUID = -3096987654278064670L; - - /** Total container memory, in bytes */ + + /** Total container memory, in bytes. */ private final long totalContainerMemoryMB; - /** Heap size to be used for the Java process */ + /** Heap size to be used for the Java process. */ private final long taskManagerHeapSizeMB; - /** Direct memory limit for the Java process */ + /** Direct memory limit for the Java process. */ private final long taskManagerDirectMemoryLimitMB; - /** The number of slots per TaskManager */ + /** The number of slots per TaskManager. */ private final int numSlots; - - /** Environment variables to add to the Java process */ + + /** Environment variables to add to the Java process. */ private final HashMap taskManagerEnv; - public ContaineredTaskManagerParameters( long totalContainerMemoryMB, long taskManagerHeapSizeMB, @@ -62,7 +61,7 @@ public ContaineredTaskManagerParameters( this.numSlots = numSlots; this.taskManagerEnv = taskManagerEnv; } - + // ------------------------------------------------------------------------ public long taskManagerTotalMemoryMB() { @@ -87,7 +86,7 @@ public Map taskManagerEnv() { // ------------------------------------------------------------------------ - + @Override public String toString() { return "TaskManagerParameters {" + @@ -104,7 +103,7 @@ public String toString() { // ------------------------------------------------------------------------ /** - * calcuate cutoff memory size used by container, it will throw an {@link IllegalArgumentException} + * Calcuate cutoff memory size used by container, it will throw an {@link IllegalArgumentException} * if the config is invalid or return the cutoff value if valid. * * @param config The Flink configuration. @@ -151,8 +150,9 @@ public static long calculateCutoffMB(Configuration config, long containerMemoryM * @return The parameters to start the TaskManager processes with. */ public static ContaineredTaskManagerParameters create( - Configuration config, long containerMemoryMB, int numSlots) - { + Configuration config, + long containerMemoryMB, + int numSlots) { // (1) try to compute how much memory used by container final long cutoffMB = calculateCutoffMB(config, containerMemoryMB); @@ -164,7 +164,7 @@ public static ContaineredTaskManagerParameters create( // (3) obtain the additional environment variables from the configuration final HashMap envVars = new HashMap<>(); final String prefix = ResourceManagerOptions.CONTAINERIZED_TASK_MANAGER_ENV_PREFIX; - + for (String key : config.keySet()) { if (key.startsWith(prefix) && key.length() > prefix.length()) { // remove prefix diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java index 230a9340cf8f6..8537d17c1ba8f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java @@ -98,10 +98,10 @@ public void testTotalMemoryDoesNotExceedContainerMemoryOffHeap() { * Test to guard {@link ContaineredTaskManagerParameters#calculateCutoffMB(Configuration, long)}. */ @Test - public void testCalculateCutoffMB() throws Exception { + public void testCalculateCutoffMB() { Configuration config = new Configuration(); - long containerMemoryMB = 1000; + long containerMemoryMB = 1000L; config.setFloat(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_RATIO, 0.1f); config.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 128); @@ -117,10 +117,9 @@ public void testCalculateCutoffMB() throws Exception { try { ContaineredTaskManagerParameters.calculateCutoffMB(config, containerMemoryMB); - } catch (IllegalArgumentException expected) { + fail("Expected to fail with an invalid argument exception."); + } catch (IllegalArgumentException ignored) { // we expected it. - return; } - fail(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java index d3d5444b67c97..f6e7b07e07b9f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java @@ -44,7 +44,7 @@ public class TaskManagerServicesTest extends TestLogger { */ @SuppressWarnings("deprecation") @Test - public void calculateNetworkBufOld() throws Exception { + public void calculateNetworkBufOld() { Configuration config = new Configuration(); config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 1); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index eab5e39f2cffb..caf7a7614c7f2 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -413,23 +413,23 @@ public void terminateCluster(ApplicationId applicationId) throws FlinkException /** * Method to validate cluster specification before deploy it, it will throw - * an {@link IllegalConfigurationException} if the {@link ClusterSpecification} is invalid. + * an {@link FlinkException} if the {@link ClusterSpecification} is invalid. + * + * @param clusterSpecification cluster specification to check against the configuration of the + * AbstractYarnClusterDescriptor + * @throws FlinkException if the cluster cannot be started with the provided {@link ClusterSpecification} */ - private void validateClusterSpecification(ClusterSpecification clusterSpecification) { - long taskManagerMemorySize = clusterSpecification.getTaskManagerMemoryMB(); - long cutoff; - try { - // We do the validation by calling the calculation methods here - cutoff = ContaineredTaskManagerParameters.calculateCutoffMB(flinkConfiguration, taskManagerMemorySize); - } catch (IllegalArgumentException cutoffConfigurationInvalidEx) { - throw new IllegalConfigurationException("Configurations related to cutoff checked failed.", cutoffConfigurationInvalidEx); - } - + private void validateClusterSpecification(ClusterSpecification clusterSpecification) throws FlinkException { try { + final long taskManagerMemorySize = clusterSpecification.getTaskManagerMemoryMB(); // We do the validation by calling the calculation methods here + // Internally these methods will check whether the cluster can be started with the provided + // ClusterSpecification and the configured memory requirements + final long cutoff = ContaineredTaskManagerParameters.calculateCutoffMB(flinkConfiguration, taskManagerMemorySize); TaskManagerServices.calculateHeapSizeMB(taskManagerMemorySize - cutoff, flinkConfiguration); - } catch (IllegalArgumentException heapSizeConfigurationInvalidEx) { - throw new IllegalConfigurationException("Configurations related to heap size checked failed.", heapSizeConfigurationInvalidEx); + } catch (IllegalArgumentException iae) { + throw new FlinkException("Cannot fulfill the minimum memory requirements with the provided " + + "cluster specification. Please increase the memory of the cluster.", iae); } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 1443f9957cd35..2311e875c2f13 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -31,7 +31,6 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; -import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.clusterframework.messages.GetClusterStatusResponse; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.security.SecurityConfiguration; diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java index 5b0d42219bedc..62110eda5bfdb 100644 --- a/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java +++ b/flink-yarn/src/test/java/org/apache/flink/yarn/FlinkYarnSessionCliTest.java @@ -260,13 +260,14 @@ public void testYarnIDOverridesPropertiesFile() throws Exception { @Test public void testCommandLineClusterSpecification() throws Exception { final Configuration configuration = new Configuration(); - configuration.setInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, 1337); - configuration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, 7331); - configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); - final int jobManagerMemory = 1337; final int taskManagerMemory = 7331; final int slotsPerTaskManager = 30; + + configuration.setInteger(JobManagerOptions.JOB_MANAGER_HEAP_MEMORY, jobManagerMemory); + configuration.setInteger(TaskManagerOptions.TASK_MANAGER_HEAP_MEMORY, taskManagerMemory); + configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, slotsPerTaskManager); + final String[] args = {"-yjm", String.valueOf(jobManagerMemory), "-ytm", String.valueOf(taskManagerMemory), "-ys", String.valueOf(slotsPerTaskManager)}; final FlinkYarnSessionCli flinkYarnSessionCli = new FlinkYarnSessionCli( configuration, From 27189d8058d6c3bc00dbc8409f40bedbccf01ac5 Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Mon, 19 Mar 2018 19:48:32 +0800 Subject: [PATCH 169/268] [FLINK-9022][state] Fix resource release in StreamTaskStateInitializerImpl.streamOperatorStateContext() This closes #5716. --- .../streaming/state/RocksDBKeyedStateBackend.java | 3 ++- .../streaming/api/operators/AbstractStreamOperator.java | 4 ++-- .../api/operators/StreamTaskStateInitializerImpl.java | 9 +++------ 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 6a661210ec703..41b7bd00dbfc7 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -1620,7 +1620,6 @@ public RunnableFuture> performSnapshot( CheckpointOptions checkpointOptions) throws Exception { long startTime = System.currentTimeMillis(); - final CloseableRegistry snapshotCloseableRegistry = new CloseableRegistry(); if (kvStateInformation.isEmpty()) { if (LOG.isDebugEnabled()) { @@ -1647,6 +1646,8 @@ public RunnableFuture> performSnapshot( CheckpointedStateScope.EXCLUSIVE, primaryStreamFactory); + final CloseableRegistry snapshotCloseableRegistry = new CloseableRegistry(); + final RocksDBFullSnapshotOperation snapshotOperation = new RocksDBFullSnapshotOperation<>( RocksDBKeyedStateBackend.this, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 4d3f9f57fc77c..e447cbeec05b0 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -248,8 +248,8 @@ public final void initializeState() throws Exception { context.isRestored(), // information whether we restore or start for the first time operatorStateBackend, // access to operator state backend keyedStateStore, // access to keyed state backend - keyedStateInputs, // access to operator state stream - operatorStateInputs); // access to keyed state stream + keyedStateInputs, // access to keyed state stream + operatorStateInputs); // access to operator state stream initializeState(initializationContext); } finally { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index 7e915544e4c11..d9bd089b1bcb7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -166,11 +166,12 @@ public StreamOperatorStateContext streamOperatorStateContext( // cleanup if something went wrong before results got published. if (streamTaskCloseableRegistry.unregisterCloseable(keyedStatedBackend)) { - IOUtils.closeQuietly(keyedStatedBackend); + // release resource (e.g native resource) + keyedStatedBackend.dispose(); } if (streamTaskCloseableRegistry.unregisterCloseable(operatorStateBackend)) { - IOUtils.closeQuietly(keyedStatedBackend); + operatorStateBackend.dispose(); } if (streamTaskCloseableRegistry.unregisterCloseable(rawKeyedStateInputs)) { @@ -181,10 +182,6 @@ public StreamOperatorStateContext streamOperatorStateContext( IOUtils.closeQuietly(rawOperatorStateInputs); } - if (streamTaskCloseableRegistry.unregisterCloseable(rawOperatorStateInputs)) { - IOUtils.closeQuietly(rawOperatorStateInputs); - } - throw new Exception("Exception while creating StreamOperatorStateContext.", ex); } } From 9f2b3c33272f82c275d1a5d7bca5d8c72702f8ce Mon Sep 17 00:00:00 2001 From: vinoyang Date: Sat, 24 Feb 2018 14:50:55 +0800 Subject: [PATCH 170/268] [FLINK-8756][Client] Support ClusterClient.getAccumulators() in RestClusterClient This closes #5573. --- .../program/rest/RestClusterClient.java | 41 ++++++++++- .../program/rest/RestClusterClientTest.java | 68 +++++++++++++++++++ .../handler/job/JobAccumulatorsHandler.java | 35 +++++++--- ...sIncludeSerializedValueQueryParameter.java | 41 +++++++++++ .../rest/messages/JobAccumulatorsHeaders.java | 6 +- .../rest/messages/JobAccumulatorsInfo.java | 46 ++++++++++++- .../JobAccumulatorsMessageParameters.java | 36 ++++++++++ .../json/SerializedValueDeserializer.java | 6 ++ .../json/SerializedValueSerializer.java | 6 ++ .../messages/JobAccumulatorsInfoTest.java | 2 +- 10 files changed, 273 insertions(+), 14 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/AccumulatorsIncludeSerializedValueQueryParameter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsMessageParameters.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 5558461dbf609..f3f196182011e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; +import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.time.Time; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.ProgramInvocationException; @@ -52,6 +53,9 @@ import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.EmptyResponseBody; +import org.apache.flink.runtime.rest.messages.JobAccumulatorsHeaders; +import org.apache.flink.runtime.rest.messages.JobAccumulatorsInfo; +import org.apache.flink.runtime.rest.messages.JobAccumulatorsMessageParameters; import org.apache.flink.runtime.rest.messages.JobMessageParameters; import org.apache.flink.runtime.rest.messages.JobTerminationHeaders; import org.apache.flink.runtime.rest.messages.JobTerminationMessageParameters; @@ -101,6 +105,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -403,6 +408,40 @@ private CompletableFuture triggerSavepoint( }); } + @Override + public Map getAccumulators(final JobID jobID, ClassLoader loader) throws Exception { + final JobAccumulatorsHeaders accumulatorsHeaders = JobAccumulatorsHeaders.getInstance(); + final JobAccumulatorsMessageParameters accMsgParams = accumulatorsHeaders.getUnresolvedMessageParameters(); + accMsgParams.jobPathParameter.resolve(jobID); + accMsgParams.includeSerializedAccumulatorsParameter.resolve(Collections.singletonList(true)); + + CompletableFuture responseFuture = sendRequest( + accumulatorsHeaders, + accMsgParams + ); + + Map result = Collections.emptyMap(); + + try { + result = responseFuture.thenApply((JobAccumulatorsInfo accumulatorsInfo) -> { + try { + return AccumulatorHelper.deserializeAccumulators( + accumulatorsInfo.getSerializedUserAccumulators(), + loader); + } catch (Exception e) { + throw new CompletionException( + new FlinkException( + String.format("Deserialization of accumulators for job %s failed.", jobID), + e)); + } + }).get(timeout.toMillis(), TimeUnit.MILLISECONDS); + } catch (ExecutionException ee) { + ExceptionUtils.rethrowException(ExceptionUtils.stripExecutionException(ee)); + } + + return result; + } + private CompletableFuture pollSavepointAsync( final JobID jobId, final TriggerId triggerID) { @@ -661,7 +700,7 @@ private CompletableFuture getDispatcherAddress() { TimeUnit.MILLISECONDS) .thenApplyAsync(leaderAddressSessionId -> { final String address = leaderAddressSessionId.f0; - final Optional host = ScalaUtils.toJava(AddressFromURIString.parse(address).host()); + final Optional host = ScalaUtils.toJava(AddressFromURIString.parse(address).host()); return host.orElseGet(() -> { // if the dispatcher address does not contain a host part, then assume it's running diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index ca2ba223ebeef..e108a0b116eb3 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -47,11 +47,15 @@ import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationResult; import org.apache.flink.runtime.rest.handler.async.TriggerResponse; +import org.apache.flink.runtime.rest.messages.AccumulatorsIncludeSerializedValueQueryParameter; import org.apache.flink.runtime.rest.messages.BlobServerPortHeaders; import org.apache.flink.runtime.rest.messages.BlobServerPortResponseBody; import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.EmptyResponseBody; +import org.apache.flink.runtime.rest.messages.JobAccumulatorsHeaders; +import org.apache.flink.runtime.rest.messages.JobAccumulatorsInfo; +import org.apache.flink.runtime.rest.messages.JobAccumulatorsMessageParameters; import org.apache.flink.runtime.rest.messages.JobMessageParameters; import org.apache.flink.runtime.rest.messages.JobTerminationHeaders; import org.apache.flink.runtime.rest.messages.JobTerminationMessageParameters; @@ -102,8 +106,10 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -118,6 +124,7 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -542,6 +549,67 @@ public void testListJobs() throws Exception { } } + @Test + public void testGetAccumulators() throws Exception { + TestAccumulatorHandler accumulatorHandler = new TestAccumulatorHandler(); + + try (TestRestServerEndpoint ignored = createRestServerEndpoint(accumulatorHandler)){ + + JobID id = new JobID(); + + { + Map accumulators = restClusterClient.getAccumulators(id); + assertNotNull(accumulators); + assertEquals(1, accumulators.size()); + + assertEquals(true, accumulators.containsKey("testKey")); + assertEquals("testValue", accumulators.get("testKey").toString()); + } + } + } + + private class TestAccumulatorHandler extends TestHandler { + + public TestAccumulatorHandler() { + super(JobAccumulatorsHeaders.getInstance()); + } + + @Override + protected CompletableFuture handleRequest( + @Nonnull HandlerRequest request, + @Nonnull DispatcherGateway gateway) throws RestHandlerException { + JobAccumulatorsInfo accumulatorsInfo; + List queryParams = request.getQueryParameter(AccumulatorsIncludeSerializedValueQueryParameter.class); + + final boolean includeSerializedValue; + if (!queryParams.isEmpty()) { + includeSerializedValue = queryParams.get(0); + } else { + includeSerializedValue = false; + } + + List userTaskAccumulators = new ArrayList<>(1); + + userTaskAccumulators.add(new JobAccumulatorsInfo.UserTaskAccumulator("testName", "testType", "testValue")); + + if (includeSerializedValue) { + Map> serializedUserTaskAccumulators = new HashMap<>(1); + try { + serializedUserTaskAccumulators.put("testKey", new SerializedValue<>("testValue")); + } catch (IOException e) { + throw new RuntimeException(e); + } + + accumulatorsInfo = new JobAccumulatorsInfo(Collections.emptyList(), userTaskAccumulators, serializedUserTaskAccumulators); + } else { + accumulatorsInfo = new JobAccumulatorsInfo(Collections.emptyList(), userTaskAccumulators, Collections.emptyMap()); + } + + return CompletableFuture.completedFuture(accumulatorsInfo); + } + } + private class TestListJobsHandler extends TestHandler { private TestListJobsHandler() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobAccumulatorsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobAccumulatorsHandler.java index 7dd5ff0718690..0fe920171dcfa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobAccumulatorsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobAccumulatorsHandler.java @@ -24,12 +24,14 @@ import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; +import org.apache.flink.runtime.rest.messages.AccumulatorsIncludeSerializedValueQueryParameter; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.JobAccumulatorsInfo; -import org.apache.flink.runtime.rest.messages.JobMessageParameters; +import org.apache.flink.runtime.rest.messages.JobAccumulatorsMessageParameters; import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.SerializedValue; import java.util.ArrayList; import java.util.Collections; @@ -41,14 +43,14 @@ /** * Request handler that returns the aggregated accumulators of a job. */ -public class JobAccumulatorsHandler extends AbstractExecutionGraphHandler { +public class JobAccumulatorsHandler extends AbstractExecutionGraphHandler { public JobAccumulatorsHandler( CompletableFuture localRestAddress, GatewayRetriever leaderRetriever, Time timeout, Map responseHeaders, - MessageHeaders messageHeaders, + MessageHeaders messageHeaders, ExecutionGraphCache executionGraphCache, Executor executor) { super( @@ -62,11 +64,21 @@ public JobAccumulatorsHandler( } @Override - protected JobAccumulatorsInfo handleRequest(HandlerRequest request, AccessExecutionGraph graph) throws RestHandlerException { - StringifiedAccumulatorResult[] accs = graph.getAccumulatorResultsStringified(); - List userTaskAccumulators = new ArrayList<>(accs.length); + protected JobAccumulatorsInfo handleRequest(HandlerRequest request, AccessExecutionGraph graph) throws RestHandlerException { + JobAccumulatorsInfo accumulatorsInfo; + List queryParams = request.getQueryParameter(AccumulatorsIncludeSerializedValueQueryParameter.class); - for (StringifiedAccumulatorResult acc : accs) { + final boolean includeSerializedValue; + if (!queryParams.isEmpty()) { + includeSerializedValue = queryParams.get(0); + } else { + includeSerializedValue = false; + } + + StringifiedAccumulatorResult[] stringifiedAccs = graph.getAccumulatorResultsStringified(); + List userTaskAccumulators = new ArrayList<>(stringifiedAccs.length); + + for (StringifiedAccumulatorResult acc : stringifiedAccs) { userTaskAccumulators.add( new JobAccumulatorsInfo.UserTaskAccumulator( acc.getName(), @@ -74,6 +86,13 @@ protected JobAccumulatorsInfo handleRequest(HandlerRequest> serializedUserTaskAccumulators = graph.getAccumulatorsSerialized(); + accumulatorsInfo = new JobAccumulatorsInfo(Collections.emptyList(), userTaskAccumulators, serializedUserTaskAccumulators); + } else { + accumulatorsInfo = new JobAccumulatorsInfo(Collections.emptyList(), userTaskAccumulators, Collections.emptyMap()); + } + + return accumulatorsInfo; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/AccumulatorsIncludeSerializedValueQueryParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/AccumulatorsIncludeSerializedValueQueryParameter.java new file mode 100644 index 0000000000000..1f685c215b585 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/AccumulatorsIncludeSerializedValueQueryParameter.java @@ -0,0 +1,41 @@ +/* + * 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.rest.messages; + +/** + * Query parameter for job's accumulator handler {@link org.apache.flink.runtime.rest.handler.job.JobAccumulatorsHandler}. + */ +public class AccumulatorsIncludeSerializedValueQueryParameter extends MessageQueryParameter { + + private static final String key = "includeSerializedValue"; + + public AccumulatorsIncludeSerializedValueQueryParameter() { + super(key, MessageParameterRequisiteness.OPTIONAL); + } + + @Override + public String convertValueToString(Boolean value) { + return String.valueOf(value); + } + + @Override + public Boolean convertStringToValue(String value) { + return Boolean.valueOf(value); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsHeaders.java index 00f4fd5d67833..2e00c91cbb2a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsHeaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsHeaders.java @@ -26,7 +26,7 @@ /** * Message headers for the {@link JobAccumulatorsHandler}. */ -public class JobAccumulatorsHeaders implements MessageHeaders { +public class JobAccumulatorsHeaders implements MessageHeaders { private static final JobAccumulatorsHeaders INSTANCE = new JobAccumulatorsHeaders(); @@ -53,8 +53,8 @@ public HttpResponseStatus getResponseStatusCode() { } @Override - public JobMessageParameters getUnresolvedMessageParameters() { - return new JobMessageParameters(); + public JobAccumulatorsMessageParameters getUnresolvedMessageParameters() { + return new JobAccumulatorsMessageParameters(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfo.java index 367a38bba87a1..22621204a7f9a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfo.java @@ -19,12 +19,19 @@ package org.apache.flink.runtime.rest.messages; import org.apache.flink.runtime.rest.handler.job.JobAccumulatorsHandler; +import org.apache.flink.runtime.rest.messages.json.SerializedValueDeserializer; +import org.apache.flink.runtime.rest.messages.json.SerializedValueSerializer; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedValue; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -33,6 +40,7 @@ public class JobAccumulatorsInfo implements ResponseBody { public static final String FIELD_NAME_JOB_ACCUMULATORS = "job-accumulators"; public static final String FIELD_NAME_USER_TASK_ACCUMULATORS = "user-task-accumulators"; + public static final String FIELD_NAME_SERIALIZED_USER_TASK_ACCUMULATORS = "serialized-user-task-accumulators"; @JsonProperty(FIELD_NAME_JOB_ACCUMULATORS) private List jobAccumulators; @@ -40,12 +48,33 @@ public class JobAccumulatorsInfo implements ResponseBody { @JsonProperty(FIELD_NAME_USER_TASK_ACCUMULATORS) private List userAccumulators; + @JsonProperty(FIELD_NAME_SERIALIZED_USER_TASK_ACCUMULATORS) + @JsonSerialize(contentUsing = SerializedValueSerializer.class) + private Map> serializedUserAccumulators; + @JsonCreator public JobAccumulatorsInfo( @JsonProperty(FIELD_NAME_JOB_ACCUMULATORS) List jobAccumulators, - @JsonProperty(FIELD_NAME_USER_TASK_ACCUMULATORS) List userAccumulators) { + @JsonProperty(FIELD_NAME_USER_TASK_ACCUMULATORS) List userAccumulators, + @JsonDeserialize(contentUsing = SerializedValueDeserializer.class) @JsonProperty(FIELD_NAME_SERIALIZED_USER_TASK_ACCUMULATORS) Map> serializedUserAccumulators) { this.jobAccumulators = Preconditions.checkNotNull(jobAccumulators); this.userAccumulators = Preconditions.checkNotNull(userAccumulators); + this.serializedUserAccumulators = Preconditions.checkNotNull(serializedUserAccumulators); + } + + @JsonIgnore + public List getJobAccumulators() { + return jobAccumulators; + } + + @JsonIgnore + public List getUserAccumulators() { + return userAccumulators; + } + + @JsonIgnore + public Map> getSerializedUserAccumulators() { + return serializedUserAccumulators; } @Override @@ -104,6 +133,21 @@ public UserTaskAccumulator( this.value = Preconditions.checkNotNull(value); } + @JsonIgnore + public String getName() { + return name; + } + + @JsonIgnore + public String getType() { + return type; + } + + @JsonIgnore + public String getValue() { + return value; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsMessageParameters.java new file mode 100644 index 0000000000000..ef235601785a4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsMessageParameters.java @@ -0,0 +1,36 @@ +/* + * 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.rest.messages; + +import java.util.Collection; +import java.util.Collections; + +/** + * Request parameter for job accumulator's handler {@link org.apache.flink.runtime.rest.handler.job.JobAccumulatorsHandler}. + */ +public class JobAccumulatorsMessageParameters extends JobMessageParameters { + + public final AccumulatorsIncludeSerializedValueQueryParameter + includeSerializedAccumulatorsParameter = new AccumulatorsIncludeSerializedValueQueryParameter(); + + @Override + public Collection> getQueryParameters() { + return Collections.singleton(includeSerializedAccumulatorsParameter); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedValueDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedValueDeserializer.java index 6a2eadb15ed3e..d7c321dc8833f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedValueDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedValueDeserializer.java @@ -21,9 +21,11 @@ import org.apache.flink.util.SerializedValue; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JavaType; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.type.TypeFactory; import java.io.IOException; @@ -34,6 +36,10 @@ public class SerializedValueDeserializer extends StdDeserializer>() {})); + } + public SerializedValueDeserializer(final JavaType valueType) { super(valueType); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedValueSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedValueSerializer.java index 0383d99f56db8..b63b1ef909191 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedValueSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/SerializedValueSerializer.java @@ -21,9 +21,11 @@ import org.apache.flink.util.SerializedValue; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JavaType; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.type.TypeFactory; import java.io.IOException; @@ -36,6 +38,10 @@ public class SerializedValueSerializer extends StdSerializer> private static final long serialVersionUID = 1L; + public SerializedValueSerializer() { + super(TypeFactory.defaultInstance().constructType(new TypeReference>() {})); + } + public SerializedValueSerializer(final JavaType javaType) { super(javaType); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfoTest.java index baaa551caef55..e0e9649b503c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfoTest.java @@ -47,6 +47,6 @@ protected JobAccumulatorsInfo getTestResponseInstance() throws Exception { "uta3.type", "uta3.value")); - return new JobAccumulatorsInfo(Collections.emptyList(), userAccumulatorList); + return new JobAccumulatorsInfo(Collections.emptyList(), userAccumulatorList, Collections.EMPTY_MAP); } } From 2b85b464d52942541b5c6d63fddd70a18a36ab9e Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 19 Mar 2018 09:48:15 +0100 Subject: [PATCH 171/268] [FLINK-8073][kafka-tests] Disable timeout in tests To get stacktraces in case of deadlock do not timeout tests programatically. This closes #5718. --- .../kafka/FlinkKafkaProducer011ITCase.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java index 81bf0bf2db441..361f269691905 100644 --- a/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java +++ b/flink-connectors/flink-connector-kafka-0.11/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer011ITCase.java @@ -80,7 +80,7 @@ public void before() { * This test ensures that transactions reusing transactional.ids (after returning to the pool) will not clash * with previous transactions using same transactional.ids. */ - @Test(timeout = 120_000L) + @Test public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exception { String topic = "flink-kafka-producer-fail-before-notify"; @@ -123,7 +123,7 @@ public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exceptio } } - @Test(timeout = 120_000L) + @Test public void testFlinkKafkaProducer011FailBeforeNotify() throws Exception { String topic = "flink-kafka-producer-fail-before-notify"; @@ -166,7 +166,7 @@ public void testFlinkKafkaProducer011FailBeforeNotify() throws Exception { deleteTestTopic(topic); } - @Test(timeout = 120_000L) + @Test public void testFlinkKafkaProducer011FailTransactionCoordinatorBeforeNotify() throws Exception { String topic = "flink-kafka-producer-fail-transaction-coordinator-before-notify"; @@ -221,7 +221,7 @@ public void testFlinkKafkaProducer011FailTransactionCoordinatorBeforeNotify() th * If such transactions were left alone lingering it consumers would be unable to read committed records * that were created after this lingering transaction. */ - @Test(timeout = 120_000L) + @Test public void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception { String topic = "flink-kafka-producer-fail-before-notify"; @@ -263,7 +263,7 @@ public void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception { deleteTestTopic(topic); } - @Test(timeout = 120_000L) + @Test public void testFailAndRecoverSameCheckpointTwice() throws Exception { String topic = "flink-kafka-producer-fail-and-recover-same-checkpoint-twice"; @@ -316,7 +316,7 @@ public void testFailAndRecoverSameCheckpointTwice() throws Exception { * If such transactions were left alone lingering it consumers would be unable to read committed records * that were created after this lingering transaction. */ - @Test(timeout = 120_000L) + @Test public void testScaleDownBeforeFirstCheckpoint() throws Exception { String topic = "scale-down-before-first-checkpoint"; @@ -381,7 +381,7 @@ public void testScaleDownBeforeFirstCheckpoint() throws Exception { * new subtask have to generate new id(s), but he can not use ids that are potentially in use, so it has to generate * new ones that are greater then 4. */ - @Test(timeout = 120_000L) + @Test public void testScaleUpAfterScalingDown() throws Exception { String topic = "scale-down-before-first-checkpoint"; From 29fd6ce316d26c85647cc2b341fef20d983b420f Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Tue, 20 Mar 2018 11:17:18 +0100 Subject: [PATCH 172/268] [hotfix][kafka-tests] Clean up and drop unused field in KafkaProducerTestBase --- .../connectors/kafka/KafkaProducerTestBase.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 8104d8fa0f783..9278b67af0582 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -482,8 +482,7 @@ private static class BrokerRestartingMapper extends RichMapFunction private static final long serialVersionUID = 6334389850158707313L; - public static volatile boolean restartedLeaderBefore; - public static volatile boolean hasBeenCheckpointedBeforeFailure; + public static volatile boolean triggeredShutdown; public static volatile int numElementsBeforeSnapshot; public static volatile Runnable shutdownAction; @@ -491,11 +490,9 @@ private static class BrokerRestartingMapper extends RichMapFunction private int numElementsTotal; private boolean failer; - private boolean hasBeenCheckpointed; public static void resetState(Runnable shutdownAction) { - restartedLeaderBefore = false; - hasBeenCheckpointedBeforeFailure = false; + triggeredShutdown = false; numElementsBeforeSnapshot = 0; BrokerRestartingMapper.shutdownAction = shutdownAction; } @@ -513,13 +510,12 @@ public void open(Configuration parameters) { public T map(T value) throws Exception { numElementsTotal++; - if (!restartedLeaderBefore) { + if (!triggeredShutdown) { Thread.sleep(10); if (failer && numElementsTotal >= failCount) { // shut down a Kafka broker - hasBeenCheckpointedBeforeFailure = hasBeenCheckpointed; - restartedLeaderBefore = true; + triggeredShutdown = true; shutdownAction.run(); } } @@ -528,7 +524,6 @@ public T map(T value) throws Exception { @Override public void notifyCheckpointComplete(long checkpointId) { - hasBeenCheckpointed = true; } @Override From 0fa76e50f06e84da8764089f36227ed26a2c2765 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Tue, 20 Mar 2018 11:23:35 +0100 Subject: [PATCH 173/268] [FLINK-7343][kafka-tests] Fix test at-least-once test instability Previously we could set numElementsBeforeSnapshot to some value during checkpointing AFTER executing shutdown, while at the same time FlinkKafkaProducerXXX snapshot for this value would fail. This lead to incorrectly cacluated expected set of values to be present in the test kafka topic. Fix is to remember lastSnapshotedElementBeforeShutdown - last snapshot that we expect to succeed without failure. This closes #5729. --- .../kafka/Kafka09ProducerITCase.java | 11 --------- .../kafka/KafkaProducerTestBase.java | 23 +++++++++---------- 2 files changed, 11 insertions(+), 23 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java index c619c3e2e02c3..f145e56a42538 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09ProducerITCase.java @@ -32,15 +32,4 @@ public void testExactlyOnceRegularSink() throws Exception { public void testExactlyOnceCustomOperator() throws Exception { // Kafka09 does not support exactly once semantic } - - @Override - public void testOneToOneAtLeastOnceRegularSink() throws Exception { - // For some reasons this test is sometimes failing in Kafka09 while the same code works in Kafka010. Disabling - // this test because everything indicates those failures might be caused by unfixed bugs in Kafka 0.9 branch - } - - @Override - public void testOneToOneAtLeastOnceCustomOperator() throws Exception { - // Disable this test since FlinkKafka09Producer doesn't support custom operator mode - } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 9278b67af0582..5023a7eae719b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -292,7 +292,7 @@ public int partition(Integer record, byte[] key, byte[] value, String targetTopi properties, topic, partition, - Collections.unmodifiableSet(new HashSet<>(getIntegersSequence(BrokerRestartingMapper.numElementsBeforeSnapshot))), + Collections.unmodifiableSet(new HashSet<>(getIntegersSequence(BrokerRestartingMapper.lastSnapshotedElementBeforeShutdown))), KAFKA_READ_TIMEOUT); deleteTestTopic(topic); @@ -483,7 +483,7 @@ private static class BrokerRestartingMapper extends RichMapFunction private static final long serialVersionUID = 6334389850158707313L; public static volatile boolean triggeredShutdown; - public static volatile int numElementsBeforeSnapshot; + public static volatile int lastSnapshotedElementBeforeShutdown; public static volatile Runnable shutdownAction; private final int failCount; @@ -493,7 +493,7 @@ private static class BrokerRestartingMapper extends RichMapFunction public static void resetState(Runnable shutdownAction) { triggeredShutdown = false; - numElementsBeforeSnapshot = 0; + lastSnapshotedElementBeforeShutdown = 0; BrokerRestartingMapper.shutdownAction = shutdownAction; } @@ -509,15 +509,12 @@ public void open(Configuration parameters) { @Override public T map(T value) throws Exception { numElementsTotal++; + Thread.sleep(10); - if (!triggeredShutdown) { - Thread.sleep(10); - - if (failer && numElementsTotal >= failCount) { - // shut down a Kafka broker - triggeredShutdown = true; - shutdownAction.run(); - } + if (!triggeredShutdown && failer && numElementsTotal >= failCount) { + // shut down a Kafka broker + triggeredShutdown = true; + shutdownAction.run(); } return value; } @@ -528,7 +525,9 @@ public void notifyCheckpointComplete(long checkpointId) { @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { - numElementsBeforeSnapshot = numElementsTotal; + if (!triggeredShutdown) { + lastSnapshotedElementBeforeShutdown = numElementsTotal; + } } @Override From 23005ee1f653a3fcddac0f710d56f3e0d6157a48 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Fri, 16 Mar 2018 13:28:08 +0100 Subject: [PATCH 174/268] [FLINK-8984][network] Drop taskmanager.exactly-once.blocking.data.enabled config option Previously there were twe options: taskmanager.network.credit-based-flow-control.enabled and taskmanager.exactly-once.blocking.data.enabled If we disabled first one, but keept default value for the second one deadlocks will occur. By dropping taskmanager.exactly-once.blocking.data.enabled we can always use: - blocking BarrierBuffer for credit based flow control - spilling BarrierBuffer for non credit based flow control. This closes #5708. --- .../flink/configuration/TaskManagerOptions.java | 12 ------------ .../streaming/runtime/io/InputProcessorUtil.java | 2 +- 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java index 4e08fdaa9910a..c7b0782ba4f68 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/TaskManagerOptions.java @@ -326,18 +326,6 @@ public class TaskManagerOptions { .defaultValue(true) .withDescription("Boolean flag to enable/disable network credit-based flow control."); - /** - * Config parameter defining whether to spill data for channels with barrier or not in exactly-once - * mode based on credit-based flow control. - * - * @deprecated Will be removed for Flink 1.6 when the old code will be dropped in favour of - * credit-based flow control. - */ - @Deprecated - public static final ConfigOption EXACTLY_ONCE_BLOCKING_DATA_ENABLED = - key("taskmanager.exactly-once.blocking.data.enabled") - .defaultValue(true); - // ------------------------------------------------------------------------ // Task Options // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java index cb56eeefac47c..1ae34b349772a 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java @@ -51,7 +51,7 @@ public static CheckpointBarrierHandler createCheckpointBarrierHandler( + " must be positive or -1 (infinite)"); } - if (taskManagerConfig.getBoolean(TaskManagerOptions.EXACTLY_ONCE_BLOCKING_DATA_ENABLED)) { + if (taskManagerConfig.getBoolean(TaskManagerOptions.NETWORK_CREDIT_BASED_FLOW_CONTROL_ENABLED)) { barrierHandler = new BarrierBuffer(inputGate, new CachedBufferBlocker(inputGate.getPageSize()), maxAlign); } else { barrierHandler = new BarrierBuffer(inputGate, new BufferSpiller(ioManager, inputGate.getPageSize()), maxAlign); From 16b90976cb05dcb5683c3bfed34ff3ec04e7416d Mon Sep 17 00:00:00 2001 From: Florian Schmidt Date: Mon, 19 Mar 2018 13:26:14 +0100 Subject: [PATCH 175/268] [FLINK-9020][E2ETests] Use separate modules per testcase This closes #5717. --- .../pom.xml | 108 ++++++++++++++++++ .../runtime/taskmanager/TaskManager.java | 0 .../tests/ClassLoaderTestProgram.java | 0 .../src/main/resources/.version.properties | 0 flink-end-to-end-tests/pom.xml | 78 +------------ .../test_streaming_classloader.sh | 4 +- 6 files changed, 116 insertions(+), 74 deletions(-) create mode 100644 flink-end-to-end-tests/flink-parent-child-classloading-test/pom.xml rename flink-end-to-end-tests/{ => flink-parent-child-classloading-test}/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java (100%) rename flink-end-to-end-tests/{ => flink-parent-child-classloading-test}/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java (100%) rename flink-end-to-end-tests/{ => flink-parent-child-classloading-test}/src/main/resources/.version.properties (100%) diff --git a/flink-end-to-end-tests/flink-parent-child-classloading-test/pom.xml b/flink-end-to-end-tests/flink-parent-child-classloading-test/pom.xml new file mode 100644 index 0000000000000..a7d09c0380e08 --- /dev/null +++ b/flink-end-to-end-tests/flink-parent-child-classloading-test/pom.xml @@ -0,0 +1,108 @@ + + + + + + flink-end-to-end-tests + org.apache.flink + 1.5-SNAPSHOT + .. + + + 4.0.0 + + flink-parent-child-classloading-test_${scala.binary.version} + flink-parent-child-classloading-test + jar + + + + org.apache.flink + flink-core + ${project.version} + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + + ClassLoaderTestProgram + package + + jar + + + ClassLoaderTestProgram + + + + org.apache.flink.streaming.tests.ClassLoaderTestProgram + + + + + org/apache/flink/streaming/tests/ClassLoaderTestProgram.class + org/apache/flink/runtime/taskmanager/TaskManager.class + .version.properties + + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + 1.7 + + + rename + package + + run + + + + + + + + + + + + + + diff --git a/flink-end-to-end-tests/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-end-to-end-tests/flink-parent-child-classloading-test/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java similarity index 100% rename from flink-end-to-end-tests/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java rename to flink-end-to-end-tests/flink-parent-child-classloading-test/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java diff --git a/flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java b/flink-end-to-end-tests/flink-parent-child-classloading-test/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java similarity index 100% rename from flink-end-to-end-tests/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java rename to flink-end-to-end-tests/flink-parent-child-classloading-test/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java diff --git a/flink-end-to-end-tests/src/main/resources/.version.properties b/flink-end-to-end-tests/flink-parent-child-classloading-test/src/main/resources/.version.properties similarity index 100% rename from flink-end-to-end-tests/src/main/resources/.version.properties rename to flink-end-to-end-tests/flink-parent-child-classloading-test/src/main/resources/.version.properties diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 6e9dee6775747..695a4f1324d85 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -29,79 +29,13 @@ under the License. .. - flink-end-to-end-tests_${scala.binary.version} - flink-end-to-end-tests - - jar - - - - org.apache.flink - flink-core - ${project.version} - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${project.version} - - - - - - - org.apache.maven.plugins - maven-jar-plugin - 2.4 + pom - - - - ClassLoaderTestProgram - package - - jar - - - ClassLoaderTestProgram - - - - org.apache.flink.streaming.tests.ClassLoaderTestProgram - - - - - org/apache/flink/streaming/tests/ClassLoaderTestProgram.class - org/apache/flink/runtime/taskmanager/TaskManager.class - .version.properties - - - - - + flink-end-to-end-tests + flink-end-to-end-tests - - - org.apache.maven.plugins - maven-antrun-plugin - 1.7 - - - rename - package - - run - - - - - - - - - - - + + flink-parent-child-classloading-test + diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_classloader.sh b/flink-end-to-end-tests/test-scripts/test_streaming_classloader.sh index 95c58f8a1f584..34c55f70ccac4 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_classloader.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_classloader.sh @@ -19,7 +19,7 @@ source "$(dirname "$0")"/common.sh -TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/target/ClassLoaderTestProgram.jar +TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-parent-child-classloading-test/target/ClassLoaderTestProgram.jar echo "Testing parent-first class loading" @@ -111,4 +111,4 @@ if [[ "$OUTPUT" != "$EXPECTED" ]]; then echo -e "EXPECTED: $EXPECTED" echo -e "ACTUAL: $OUTPUT" PASS="" -fi \ No newline at end of file +fi From 0e2bf7eea9e2b1d694711a33662b0d5f64d0e10f Mon Sep 17 00:00:00 2001 From: Gabor Gevay Date: Tue, 13 Feb 2018 18:09:05 +0100 Subject: [PATCH 176/268] [FLINK-8649] [scala api] Pass on TypeInfo in StreamExecutionEnvironment.createInput This closes #5478. --- .../streaming/api/scala/StreamExecutionEnvironment.scala | 7 ++++++- .../apache/flink/streaming/api/scala/DataStreamTest.scala | 7 +++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala index cd96dbf8ea044..9410a95dea2d3 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala @@ -23,6 +23,7 @@ import org.apache.flink.annotation.{Internal, Public, PublicEvolving} import org.apache.flink.api.common.io.{FileInputFormat, FilePathFilter, InputFormat} import org.apache.flink.api.common.restartstrategy.RestartStrategies.RestartStrategyConfiguration import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer import org.apache.flink.api.scala.ClosureCleaner import org.apache.flink.configuration.Configuration @@ -594,7 +595,11 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) { */ @PublicEvolving def createInput[T: TypeInformation](inputFormat: InputFormat[T, _]): DataStream[T] = - asScalaStream(javaEnv.createInput(inputFormat)) + if (inputFormat.isInstanceOf[ResultTypeQueryable[_]]) { + asScalaStream(javaEnv.createInput(inputFormat)) + } else { + asScalaStream(javaEnv.createInput(inputFormat, implicitly[TypeInformation[T]])) + } /** * Create a DataStream using a user defined source function for arbitrary diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala index 51ec5e382307a..9e1c49393654a 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala @@ -21,6 +21,7 @@ package org.apache.flink.streaming.api.scala import java.lang import org.apache.flink.api.common.functions._ +import org.apache.flink.api.java.io.ParallelIteratorInputFormat import org.apache.flink.api.java.typeutils.TypeExtractor import org.apache.flink.streaming.api.collector.selector.OutputSelector import org.apache.flink.streaming.api.functions.{KeyedProcessFunction, ProcessFunction} @@ -673,6 +674,12 @@ class DataStreamTest extends AbstractTestBase { assert(sg.getIterationSourceSinkPairs.size() == 2) } + @Test + def testCreateInputPassesOnTypeInfo(): Unit = { + StreamExecutionEnvironment.getExecutionEnvironment.createInput[Tuple1[Integer]]( + new ParallelIteratorInputFormat[Tuple1[Integer]](null)) + } + ///////////////////////////////////////////////////////////// // Utilities ///////////////////////////////////////////////////////////// From c12e21cc8cb8b809689cc35c66e5363723352e22 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 21 Mar 2018 20:59:05 +0100 Subject: [PATCH 177/268] Revert "[FLINK-8703][tests] Port AutoParallelismITCase to flip6" The test does not actually run on Flip6, see FLINK-8813. --- .../test/misc/AutoParallelismITCase.java | 42 ++++++++++++++----- 1 file changed, 32 insertions(+), 10 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java index c25dbf060db7d..9cafee624f0a2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java @@ -22,15 +22,17 @@ import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.api.common.io.GenericInputFormat; import org.apache.flink.api.java.DataSet; -import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; -import org.apache.flink.test.util.MiniClusterResource; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.test.util.TestEnvironment; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; -import org.junit.ClassRule; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; @@ -50,17 +52,37 @@ public class AutoParallelismITCase extends TestLogger { private static final int SLOTS_PER_TM = 7; private static final int PARALLELISM = NUM_TM * SLOTS_PER_TM; - @ClassRule - public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( - new MiniClusterResource.MiniClusterResourceConfiguration( - new Configuration(), - 2, - 7)); + private static LocalFlinkMiniCluster cluster; + + private static TestEnvironment env; + + @BeforeClass + public static void setupCluster() { + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TM); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, SLOTS_PER_TM); + cluster = new LocalFlinkMiniCluster(config, false); + + cluster.start(); + + env = new TestEnvironment(cluster, NUM_TM * SLOTS_PER_TM, false); + } + + @AfterClass + public static void teardownCluster() { + try { + cluster.stop(); + } + catch (Throwable t) { + System.err.println("Error stopping cluster on shutdown"); + t.printStackTrace(); + fail("ClusterClient shutdown caused an exception: " + t.getMessage()); + } + } @Test public void testProgramWithAutoParallelism() { try { - ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX); env.getConfig().disableSysoutLogging(); From f0e6ae6ebfc8eaf1a0a6225b868db0f6fc0f5e60 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 21 Mar 2018 15:01:55 +0100 Subject: [PATCH 178/268] [FLINK-8925][tests] Enable flip6 on travis --- .travis.yml | 20 +++++--------------- pom.xml | 7 +++++-- 2 files changed, 10 insertions(+), 17 deletions(-) diff --git a/.travis.yml b/.travis.yml index f84b8d8b267d7..f86bbef1638c0 100644 --- a/.travis.yml +++ b/.travis.yml @@ -39,27 +39,27 @@ matrix: - jdk: "oraclejdk8" env: - TEST="core" - - PROFILE="-Dhadoop.version=2.8.3" + - PROFILE="-Dhadoop.version=2.8.3 -Dflip6" - CACHE_NAME=JDK8_H280_CO - jdk: "oraclejdk8" env: - TEST="libraries" - - PROFILE="-Dhadoop.version=2.8.3" + - PROFILE="-Dhadoop.version=2.8.3 -Dflip6" - CACHE_NAME=JDK8_H280_L - jdk: "oraclejdk8" env: - TEST="connectors" - - PROFILE="-Dhadoop.version=2.8.3 -Pinclude-kinesis" + - PROFILE="-Dhadoop.version=2.8.3 -Dflip6 -Pinclude-kinesis" - CACHE_NAME=JDK8_H280_CN - jdk: "oraclejdk8" env: - TEST="tests" - - PROFILE="-Dhadoop.version=2.8.3" + - PROFILE="-Dhadoop.version=2.8.3 -Dflip6" - CACHE_NAME=JDK8_H280_T - jdk: "oraclejdk8" env: - TEST="misc" - - PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws" + - PROFILE="-Dhadoop.version=2.8.3 -Dflip6 -Dinclude_hadoop_aws" - CACHE_NAME=JDK8_H280_M - jdk: "openjdk8" env: @@ -86,16 +86,6 @@ matrix: - TEST="misc" - PROFILE="-Dhadoop.version=2.4.1" - CACHE_NAME=JDK8_H241_M - - jdk: "oraclejdk8" - env: - - TEST="core" - - PROFILE="-Dhadoop.version=2.8.0 -Pflip6" - - CACHE_NAME=JDK8_H280_F6_CO - - jdk: "oraclejdk8" - env: - - TEST="tests" - - PROFILE="-Dhadoop.version=2.8.0 -Pflip6" - - CACHE_NAME=JDK8_H280_F6_T git: depth: 100 diff --git a/pom.xml b/pom.xml index 02ca63b1b2514..91e2fd43be408 100644 --- a/pom.xml +++ b/pom.xml @@ -128,7 +128,6 @@ under the License. 1.3 false - org.apache.flink.testutils.category.Flip6 old flip6 From 555e80a089c7d4a5d2d69d9a2f05d606e12d2eff Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Thu, 15 Mar 2018 21:04:00 +0100 Subject: [PATCH 179/268] [FLINK-8903] [table] Fix VAR_SAMP, VAR_POP, STDEV_SAMP, STDEV_POP functions on GROUP BY windows. This closes #5706. --- .../rules/AggregateReduceFunctionsRule.java | 602 ++++++++++++++++++ .../nodes/logical/FlinkLogicalAggregate.scala | 9 +- .../logical/FlinkLogicalWindowAggregate.scala | 17 + .../table/plan/rules/FlinkRuleSets.scala | 1 + .../WindowAggregateReduceFunctionsRule.scala | 75 +++ .../runtime/aggregate/AggregateUtil.scala | 4 +- .../table/api/batch/sql/GroupWindowTest.scala | 49 ++ .../api/batch/table/GroupWindowTest.scala | 45 ++ .../api/stream/sql/GroupWindowTest.scala | 46 ++ .../api/stream/table/GroupWindowTest.scala | 45 ++ 10 files changed, 888 insertions(+), 5 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowAggregateReduceFunctionsRule.scala diff --git a/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java b/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java new file mode 100644 index 0000000000000..ce466e199c262 --- /dev/null +++ b/flink-libraries/flink-table/src/main/java/org/apache/calcite/rel/rules/AggregateReduceFunctionsRule.java @@ -0,0 +1,602 @@ +/* + * 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.calcite.rel.rules; + +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptRuleOperand; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.RelFactories; +import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.type.SqlTypeUtil; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.calcite.util.CompositeList; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.calcite.util.Util; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/* + * THIS FILE HAS BEEN COPIED FROM THE APACHE CALCITE PROJECT TO MAKE IT MORE EXTENSIBLE. + * + * We have opened an issue to port this change to Apache Calcite (CALCITE-2216). + * Once CALCITE-2216 is fixed and included in a release, we can remove the copied class. + * + * Modification: + * - Added newCalcRel() method to be able to add fields to the projection. + */ + +/** + * Planner rule that reduces aggregate functions in + * {@link org.apache.calcite.rel.core.Aggregate}s to simpler forms. + * + *

    Rewrites: + *

      + * + *
    • AVG(x) → SUM(x) / COUNT(x) + * + *
    • STDDEV_POP(x) → SQRT( + * (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) + * / COUNT(x)) + * + *
    • STDDEV_SAMP(x) → SQRT( + * (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) + * / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END) + * + *
    • VAR_POP(x) → (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) + * / COUNT(x) + * + *
    • VAR_SAMP(x) → (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) + * / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END + *
    + * + *

    Since many of these rewrites introduce multiple occurrences of simpler + * forms like {@code COUNT(x)}, the rule gathers common sub-expressions as it + * goes. + */ +public class AggregateReduceFunctionsRule extends RelOptRule { + //~ Static fields/initializers --------------------------------------------- + + /** The singleton. */ + public static final AggregateReduceFunctionsRule INSTANCE = + new AggregateReduceFunctionsRule(operand(LogicalAggregate.class, any()), + RelFactories.LOGICAL_BUILDER); + + //~ Constructors ----------------------------------------------------------- + + /** Creates an AggregateReduceFunctionsRule. */ + public AggregateReduceFunctionsRule(RelOptRuleOperand operand, + RelBuilderFactory relBuilderFactory) { + super(operand, relBuilderFactory, null); + } + + //~ Methods ---------------------------------------------------------------- + + @Override public boolean matches(RelOptRuleCall call) { + if (!super.matches(call)) { + return false; + } + Aggregate oldAggRel = (Aggregate) call.rels[0]; + return containsAvgStddevVarCall(oldAggRel.getAggCallList()); + } + + public void onMatch(RelOptRuleCall ruleCall) { + Aggregate oldAggRel = (Aggregate) ruleCall.rels[0]; + reduceAggs(ruleCall, oldAggRel); + } + + /** + * Returns whether any of the aggregates are calls to AVG, STDDEV_*, VAR_*. + * + * @param aggCallList List of aggregate calls + */ + private boolean containsAvgStddevVarCall(List aggCallList) { + for (AggregateCall call : aggCallList) { + if (isReducible(call.getAggregation().getKind())) { + return true; + } + } + return false; + } + + /** + * Returns whether the aggregate call is a reducible function + */ + private boolean isReducible(final SqlKind kind) { + if (SqlKind.AVG_AGG_FUNCTIONS.contains(kind)) { + return true; + } + switch (kind) { + case SUM: + return true; + } + return false; + } + + /** + * Reduces all calls to AVG, STDDEV_POP, STDDEV_SAMP, VAR_POP, VAR_SAMP in + * the aggregates list to. + * + *

    It handles newly generated common subexpressions since this was done + * at the sql2rel stage. + */ + private void reduceAggs( + RelOptRuleCall ruleCall, + Aggregate oldAggRel) { + RexBuilder rexBuilder = oldAggRel.getCluster().getRexBuilder(); + + List oldCalls = oldAggRel.getAggCallList(); + final int groupCount = oldAggRel.getGroupCount(); + final int indicatorCount = oldAggRel.getIndicatorCount(); + + final List newCalls = Lists.newArrayList(); + final Map aggCallMapping = Maps.newHashMap(); + + final List projList = Lists.newArrayList(); + + // pass through group key (+ indicators if present) + for (int i = 0; i < groupCount + indicatorCount; ++i) { + projList.add( + rexBuilder.makeInputRef( + getFieldType(oldAggRel, i), + i)); + } + + // List of input expressions. If a particular aggregate needs more, it + // will add an expression to the end, and we will create an extra + // project. + final RelBuilder relBuilder = ruleCall.builder(); + relBuilder.push(oldAggRel.getInput()); + final List inputExprs = new ArrayList<>(relBuilder.fields()); + + // create new agg function calls and rest of project list together + for (AggregateCall oldCall : oldCalls) { + projList.add( + reduceAgg( + oldAggRel, oldCall, newCalls, aggCallMapping, inputExprs)); + } + + final int extraArgCount = + inputExprs.size() - relBuilder.peek().getRowType().getFieldCount(); + if (extraArgCount > 0) { + relBuilder.project(inputExprs, + CompositeList.of( + relBuilder.peek().getRowType().getFieldNames(), + Collections.nCopies(extraArgCount, null))); + } + newAggregateRel(relBuilder, oldAggRel, newCalls); + newCalcRel(relBuilder, oldAggRel, projList); + ruleCall.transformTo(relBuilder.build()); + } + + private RexNode reduceAgg( + Aggregate oldAggRel, + AggregateCall oldCall, + List newCalls, + Map aggCallMapping, + List inputExprs) { + final SqlKind kind = oldCall.getAggregation().getKind(); + if (isReducible(kind)) { + switch (kind) { + case SUM: + // replace original SUM(x) with + // case COUNT(x) when 0 then null else SUM0(x) end + return reduceSum(oldAggRel, oldCall, newCalls, aggCallMapping); + case AVG: + // replace original AVG(x) with SUM(x) / COUNT(x) + return reduceAvg(oldAggRel, oldCall, newCalls, aggCallMapping, inputExprs); + case STDDEV_POP: + // replace original STDDEV_POP(x) with + // SQRT( + // (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) + // / COUNT(x)) + return reduceStddev(oldAggRel, oldCall, true, true, newCalls, + aggCallMapping, inputExprs); + case STDDEV_SAMP: + // replace original STDDEV_POP(x) with + // SQRT( + // (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) + // / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END) + return reduceStddev(oldAggRel, oldCall, false, true, newCalls, + aggCallMapping, inputExprs); + case VAR_POP: + // replace original VAR_POP(x) with + // (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) + // / COUNT(x) + return reduceStddev(oldAggRel, oldCall, true, false, newCalls, + aggCallMapping, inputExprs); + case VAR_SAMP: + // replace original VAR_POP(x) with + // (SUM(x * x) - SUM(x) * SUM(x) / COUNT(x)) + // / CASE COUNT(x) WHEN 1 THEN NULL ELSE COUNT(x) - 1 END + return reduceStddev(oldAggRel, oldCall, false, false, newCalls, + aggCallMapping, inputExprs); + default: + throw Util.unexpected(kind); + } + } else { + // anything else: preserve original call + RexBuilder rexBuilder = oldAggRel.getCluster().getRexBuilder(); + final int nGroups = oldAggRel.getGroupCount(); + List oldArgTypes = + SqlTypeUtil.projectTypes( + oldAggRel.getInput().getRowType(), oldCall.getArgList()); + return rexBuilder.addAggCall(oldCall, + nGroups, + oldAggRel.indicator, + newCalls, + aggCallMapping, + oldArgTypes); + } + } + + private AggregateCall createAggregateCallWithBinding( + RelDataTypeFactory typeFactory, + SqlAggFunction aggFunction, + RelDataType operandType, + Aggregate oldAggRel, + AggregateCall oldCall, + int argOrdinal) { + final Aggregate.AggCallBinding binding = + new Aggregate.AggCallBinding(typeFactory, aggFunction, + ImmutableList.of(operandType), oldAggRel.getGroupCount(), + oldCall.filterArg >= 0); + return AggregateCall.create(aggFunction, + oldCall.isDistinct(), + oldCall.isApproximate(), + ImmutableIntList.of(argOrdinal), + oldCall.filterArg, + aggFunction.inferReturnType(binding), + null); + } + + private RexNode reduceAvg( + Aggregate oldAggRel, + AggregateCall oldCall, + List newCalls, + Map aggCallMapping, + List inputExprs) { + final int nGroups = oldAggRel.getGroupCount(); + final RexBuilder rexBuilder = oldAggRel.getCluster().getRexBuilder(); + final int iAvgInput = oldCall.getArgList().get(0); + final RelDataType avgInputType = + getFieldType( + oldAggRel.getInput(), + iAvgInput); + final AggregateCall sumCall = + AggregateCall.create(SqlStdOperatorTable.SUM, + oldCall.isDistinct(), + oldCall.isApproximate(), + oldCall.getArgList(), + oldCall.filterArg, + oldAggRel.getGroupCount(), + oldAggRel.getInput(), + null, + null); + final AggregateCall countCall = + AggregateCall.create(SqlStdOperatorTable.COUNT, + oldCall.isDistinct(), + oldCall.isApproximate(), + oldCall.getArgList(), + oldCall.filterArg, + oldAggRel.getGroupCount(), + oldAggRel.getInput(), + null, + null); + + // NOTE: these references are with respect to the output + // of newAggRel + RexNode numeratorRef = + rexBuilder.addAggCall(sumCall, + nGroups, + oldAggRel.indicator, + newCalls, + aggCallMapping, + ImmutableList.of(avgInputType)); + final RexNode denominatorRef = + rexBuilder.addAggCall(countCall, + nGroups, + oldAggRel.indicator, + newCalls, + aggCallMapping, + ImmutableList.of(avgInputType)); + + final RelDataTypeFactory typeFactory = oldAggRel.getCluster().getTypeFactory(); + final RelDataType avgType = typeFactory.createTypeWithNullability( + oldCall.getType(), numeratorRef.getType().isNullable()); + numeratorRef = rexBuilder.ensureType(avgType, numeratorRef, true); + final RexNode divideRef = + rexBuilder.makeCall(SqlStdOperatorTable.DIVIDE, numeratorRef, denominatorRef); + return rexBuilder.makeCast(oldCall.getType(), divideRef); + } + + private RexNode reduceSum( + Aggregate oldAggRel, + AggregateCall oldCall, + List newCalls, + Map aggCallMapping) { + final int nGroups = oldAggRel.getGroupCount(); + RexBuilder rexBuilder = oldAggRel.getCluster().getRexBuilder(); + int arg = oldCall.getArgList().get(0); + RelDataType argType = + getFieldType( + oldAggRel.getInput(), + arg); + final AggregateCall sumZeroCall = + AggregateCall.create(SqlStdOperatorTable.SUM0, oldCall.isDistinct(), + oldCall.isApproximate(), oldCall.getArgList(), oldCall.filterArg, + oldAggRel.getGroupCount(), oldAggRel.getInput(), null, + oldCall.name); + final AggregateCall countCall = + AggregateCall.create(SqlStdOperatorTable.COUNT, + oldCall.isDistinct(), + oldCall.isApproximate(), + oldCall.getArgList(), + oldCall.filterArg, + oldAggRel.getGroupCount(), + oldAggRel, + null, + null); + + // NOTE: these references are with respect to the output + // of newAggRel + RexNode sumZeroRef = + rexBuilder.addAggCall(sumZeroCall, + nGroups, + oldAggRel.indicator, + newCalls, + aggCallMapping, + ImmutableList.of(argType)); + if (!oldCall.getType().isNullable()) { + // If SUM(x) is not nullable, the validator must have determined that + // nulls are impossible (because the group is never empty and x is never + // null). Therefore we translate to SUM0(x). + return sumZeroRef; + } + RexNode countRef = + rexBuilder.addAggCall(countCall, + nGroups, + oldAggRel.indicator, + newCalls, + aggCallMapping, + ImmutableList.of(argType)); + return rexBuilder.makeCall(SqlStdOperatorTable.CASE, + rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, + countRef, rexBuilder.makeExactLiteral(BigDecimal.ZERO)), + rexBuilder.makeCast(sumZeroRef.getType(), rexBuilder.constantNull()), + sumZeroRef); + } + + private RexNode reduceStddev( + Aggregate oldAggRel, + AggregateCall oldCall, + boolean biased, + boolean sqrt, + List newCalls, + Map aggCallMapping, + List inputExprs) { + // stddev_pop(x) ==> + // power( + // (sum(x * x) - sum(x) * sum(x) / count(x)) + // / count(x), + // .5) + // + // stddev_samp(x) ==> + // power( + // (sum(x * x) - sum(x) * sum(x) / count(x)) + // / nullif(count(x) - 1, 0), + // .5) + final int nGroups = oldAggRel.getGroupCount(); + final RelOptCluster cluster = oldAggRel.getCluster(); + final RexBuilder rexBuilder = cluster.getRexBuilder(); + final RelDataTypeFactory typeFactory = cluster.getTypeFactory(); + + assert oldCall.getArgList().size() == 1 : oldCall.getArgList(); + final int argOrdinal = oldCall.getArgList().get(0); + final RelDataType argOrdinalType = getFieldType(oldAggRel.getInput(), argOrdinal); + final RelDataType oldCallType = + typeFactory.createTypeWithNullability(oldCall.getType(), + argOrdinalType.isNullable()); + + final RexNode argRef = + rexBuilder.ensureType(oldCallType, inputExprs.get(argOrdinal), true); + final int argRefOrdinal = lookupOrAdd(inputExprs, argRef); + + final RexNode argSquared = rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, + argRef, argRef); + final int argSquaredOrdinal = lookupOrAdd(inputExprs, argSquared); + + final AggregateCall sumArgSquaredAggCall = + createAggregateCallWithBinding(typeFactory, SqlStdOperatorTable.SUM, + argSquared.getType(), oldAggRel, oldCall, argSquaredOrdinal); + + final RexNode sumArgSquared = + rexBuilder.addAggCall(sumArgSquaredAggCall, + nGroups, + oldAggRel.indicator, + newCalls, + aggCallMapping, + ImmutableList.of(sumArgSquaredAggCall.getType())); + + final AggregateCall sumArgAggCall = + AggregateCall.create(SqlStdOperatorTable.SUM, + oldCall.isDistinct(), + oldCall.isApproximate(), + ImmutableIntList.of(argOrdinal), + oldCall.filterArg, + oldAggRel.getGroupCount(), + oldAggRel.getInput(), + null, + null); + + final RexNode sumArg = + rexBuilder.addAggCall(sumArgAggCall, + nGroups, + oldAggRel.indicator, + newCalls, + aggCallMapping, + ImmutableList.of(sumArgAggCall.getType())); + final RexNode sumArgCast = rexBuilder.ensureType(oldCallType, sumArg, true); + final RexNode sumSquaredArg = + rexBuilder.makeCall( + SqlStdOperatorTable.MULTIPLY, sumArgCast, sumArgCast); + + final AggregateCall countArgAggCall = + AggregateCall.create(SqlStdOperatorTable.COUNT, + oldCall.isDistinct(), + oldCall.isApproximate(), + oldCall.getArgList(), + oldCall.filterArg, + oldAggRel.getGroupCount(), + oldAggRel, + null, + null); + + final RexNode countArg = + rexBuilder.addAggCall(countArgAggCall, + nGroups, + oldAggRel.indicator, + newCalls, + aggCallMapping, + ImmutableList.of(argOrdinalType)); + + final RexNode avgSumSquaredArg = + rexBuilder.makeCall( + SqlStdOperatorTable.DIVIDE, sumSquaredArg, countArg); + + final RexNode diff = + rexBuilder.makeCall( + SqlStdOperatorTable.MINUS, + sumArgSquared, avgSumSquaredArg); + + final RexNode denominator; + if (biased) { + denominator = countArg; + } else { + final RexLiteral one = + rexBuilder.makeExactLiteral(BigDecimal.ONE); + final RexNode nul = + rexBuilder.makeCast(countArg.getType(), rexBuilder.constantNull()); + final RexNode countMinusOne = + rexBuilder.makeCall( + SqlStdOperatorTable.MINUS, countArg, one); + final RexNode countEqOne = + rexBuilder.makeCall( + SqlStdOperatorTable.EQUALS, countArg, one); + denominator = + rexBuilder.makeCall( + SqlStdOperatorTable.CASE, + countEqOne, nul, countMinusOne); + } + + final RexNode div = + rexBuilder.makeCall( + SqlStdOperatorTable.DIVIDE, diff, denominator); + + RexNode result = div; + if (sqrt) { + final RexNode half = + rexBuilder.makeExactLiteral(new BigDecimal("0.5")); + result = + rexBuilder.makeCall( + SqlStdOperatorTable.POWER, div, half); + } + + return rexBuilder.makeCast( + oldCall.getType(), result); + } + + /** + * Finds the ordinal of an element in a list, or adds it. + * + * @param list List + * @param element Element to lookup or add + * @param Element type + * @return Ordinal of element in list + */ + private static int lookupOrAdd(List list, T element) { + int ordinal = list.indexOf(element); + if (ordinal == -1) { + ordinal = list.size(); + list.add(element); + } + return ordinal; + } + + /** + * Do a shallow clone of oldAggRel and update aggCalls. Could be refactored + * into Aggregate and subclasses - but it's only needed for some + * subclasses. + * + * @param relBuilder Builder of relational expressions; at the top of its + * stack is its input + * @param oldAggregate LogicalAggregate to clone. + * @param newCalls New list of AggregateCalls + */ + protected void newAggregateRel(RelBuilder relBuilder, + Aggregate oldAggregate, + List newCalls) { + relBuilder.aggregate( + relBuilder.groupKey(oldAggregate.getGroupSet(), + oldAggregate.getGroupSets()), + newCalls); + } + + /** + * Add a calc with the expressions to compute the original agg calls from the + * decomposed ones. + * + * @param relBuilder Builder of relational expressions; at the top of its + * stack is its input + * @param oldAggregate The original LogicalAggregate that is replaced. + * @param exprs The expressions to compute the original agg calls. + */ + protected void newCalcRel(RelBuilder relBuilder, + Aggregate oldAggregate, + List exprs) { + relBuilder.project(exprs, oldAggregate.getRowType().getFieldNames()); + } + + private RelDataType getFieldType(RelNode relNode, int i) { + final RelDataTypeField inputField = + relNode.getRowType().getFieldList().get(i); + return inputField.getType(); + } +} + +// End AggregateReduceFunctionsRule.java diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalAggregate.scala index e1e93c7c583b9..17b6f1b6f8756 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalAggregate.scala @@ -30,7 +30,7 @@ import org.apache.calcite.sql.SqlKind import org.apache.calcite.util.ImmutableBitSet import org.apache.flink.table.plan.nodes.FlinkConventions -import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ class FlinkLogicalAggregate( cluster: RelOptCluster, @@ -74,8 +74,11 @@ private class FlinkLogicalAggregateConverter // we do not support these functions natively // they have to be converted using the AggregateReduceFunctionsRule - val supported = agg.getAggCallList.map(_.getAggregation.getKind).forall { - case SqlKind.STDDEV_POP | SqlKind.STDDEV_SAMP | SqlKind.VAR_POP | SqlKind.VAR_SAMP => false + val supported = agg.getAggCallList.asScala.map(_.getAggregation.getKind).forall { + // we support AVG + case SqlKind.AVG => true + // but none of the other AVG agg functions + case k if SqlKind.AVG_AGG_FUNCTIONS.contains(k) => false case _ => true } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalWindowAggregate.scala index 3e605e895dceb..f2576f4c85378 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalWindowAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalWindowAggregate.scala @@ -26,6 +26,7 @@ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.core.{Aggregate, AggregateCall} import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rel.{RelNode, RelShuttle} +import org.apache.calcite.sql.SqlKind import org.apache.calcite.util.ImmutableBitSet import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.calcite.FlinkTypeFactory @@ -33,6 +34,8 @@ import org.apache.flink.table.plan.logical.LogicalWindow import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate import org.apache.flink.table.plan.nodes.FlinkConventions +import scala.collection.JavaConverters._ + class FlinkLogicalWindowAggregate( window: LogicalWindow, namedProperties: Seq[NamedWindowProperty], @@ -103,6 +106,20 @@ class FlinkLogicalWindowAggregateConverter FlinkConventions.LOGICAL, "FlinkLogicalWindowAggregateConverter") { + override def matches(call: RelOptRuleCall): Boolean = { + val agg = call.rel(0).asInstanceOf[LogicalWindowAggregate] + + // we do not support these functions natively + // they have to be converted using the WindowAggregateReduceFunctionsRule + agg.getAggCallList.asScala.map(_.getAggregation.getKind).forall { + // we support AVG + case SqlKind.AVG => true + // but none of the other AVG agg functions + case k if SqlKind.AVG_AGG_FUNCTIONS.contains(k) => false + case _ => true + } + } + override def convert(rel: RelNode): RelNode = { val agg = rel.asInstanceOf[LogicalWindowAggregate] val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala index d3ad2ac5654dc..9f3b8e99ece61 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -93,6 +93,7 @@ object FlinkRuleSets { // reduce aggregate functions like AVG, STDDEV_POP etc. AggregateReduceFunctionsRule.INSTANCE, + WindowAggregateReduceFunctionsRule.INSTANCE, // remove unnecessary sort rule SortRemoveRule.INSTANCE, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowAggregateReduceFunctionsRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowAggregateReduceFunctionsRule.scala new file mode 100644 index 0000000000000..4ca2b335478d6 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowAggregateReduceFunctionsRule.scala @@ -0,0 +1,75 @@ +/* + * 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.table.plan.rules.common + +import java.util + +import org.apache.calcite.plan.RelOptRule +import org.apache.calcite.rel.core.{Aggregate, AggregateCall, RelFactories} +import org.apache.calcite.rel.logical.LogicalAggregate +import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule +import org.apache.calcite.rex.RexNode +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate + +/** + * Rule to convert complex aggregation functions into simpler ones. + * Have a look at [[AggregateReduceFunctionsRule]] for details. + */ +class WindowAggregateReduceFunctionsRule extends AggregateReduceFunctionsRule( + RelOptRule.operand(classOf[LogicalWindowAggregate], RelOptRule.any()), + RelFactories.LOGICAL_BUILDER) { + + override def newAggregateRel( + relBuilder: RelBuilder, + oldAgg: Aggregate, + newCalls: util.List[AggregateCall]): Unit = { + + // create a LogicalAggregate with simpler aggregation functions + super.newAggregateRel(relBuilder, oldAgg, newCalls) + // pop LogicalAggregate from RelBuilder + val newAgg = relBuilder.build().asInstanceOf[LogicalAggregate] + + // create a new LogicalWindowAggregate (based on the new LogicalAggregate) and push it on the + // RelBuilder + val oldWindowAgg = oldAgg.asInstanceOf[LogicalWindowAggregate] + relBuilder.push(LogicalWindowAggregate.create( + oldWindowAgg.getWindow, + oldWindowAgg.getNamedProperties, + newAgg)) + } + + override def newCalcRel( + relBuilder: RelBuilder, + oldAgg: Aggregate, + exprs: util.List[RexNode]): Unit = { + + // add all named properties of the window to the selection + val oldWindowAgg = oldAgg.asInstanceOf[LogicalWindowAggregate] + oldWindowAgg.getNamedProperties.foreach(np => exprs.add(relBuilder.field(np.name))) + + // create a LogicalCalc that computes the complex aggregates and forwards the window properties + relBuilder.project(exprs, oldAgg.getRowType.getFieldNames) + } + +} + +object WindowAggregateReduceFunctionsRule { + val INSTANCE = new WindowAggregateReduceFunctionsRule +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala index df9b1c5520467..ce0a9c96e336e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala @@ -1259,7 +1259,7 @@ object AggregateUtil { } } - case _: SqlAvgAggFunction => + case a: SqlAvgAggFunction if a.kind == SqlKind.AVG => aggregates(index) = sqlTypeName match { case TINYINT => new ByteAvgAggFunction @@ -1413,7 +1413,7 @@ object AggregateUtil { accTypes(index) = udagg.accType case unSupported: SqlAggFunction => - throw new TableException(s"unsupported Function: '${unSupported.getName}'") + throw new TableException(s"Unsupported Function: '${unSupported.getName}'") } } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupWindowTest.scala index 8d06bcd84db39..b1369e2a9289a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupWindowTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/sql/GroupWindowTest.scala @@ -304,4 +304,53 @@ class GroupWindowTest extends TableTestBase { util.verifySql(sql, expected) } + + @Test + def testDecomposableAggFunctions() = { + val util = batchTestUtil() + util.addTable[(Int, String, Long, Timestamp)]("MyTable", 'a, 'b, 'c, 'rowtime) + + val sql = + "SELECT " + + " VAR_POP(c), VAR_SAMP(c), STDDEV_POP(c), STDDEV_SAMP(c), " + + " TUMBLE_START(rowtime, INTERVAL '15' MINUTE), " + + " TUMBLE_END(rowtime, INTERVAL '15' MINUTE)" + + "FROM MyTable " + + "GROUP BY TUMBLE(rowtime, INTERVAL '15' MINUTE)" + + val expected = + unaryNode( + "DataSetCalc", + unaryNode( + "DataSetWindowAggregate", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "rowtime", "c", + "*(c, c) AS $f2", "*(c, c) AS $f3", "*(c, c) AS $f4", "*(c, c) AS $f5") + ), + term("window", TumblingGroupWindow('w$, 'rowtime, 900000.millis)), + term("select", + "SUM($f2) AS $f0", + "SUM(c) AS $f1", + "COUNT(c) AS $f2", + "SUM($f3) AS $f3", + "SUM($f4) AS $f4", + "SUM($f5) AS $f5", + "start('w$) AS w$start", + "end('w$) AS w$end", + "rowtime('w$) AS w$rowtime") + ), + term("select", + "CAST(/(-($f0, /(*($f1, $f1), $f2)), $f2)) AS EXPR$0", + "CAST(/(-($f3, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1)))) AS EXPR$1", + "CAST(POWER(/(-($f4, /(*($f1, $f1), $f2)), $f2), 0.5)) AS EXPR$2", + "CAST(POWER(/(-($f5, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1))), 0.5)) " + + "AS EXPR$3", + "CAST(w$start) AS EXPR$4", + "CAST(w$end) AS EXPR$5") + ) + + util.verifySql(sql, expected) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/GroupWindowTest.scala index ad44e09c68aad..27c1d7f6c324a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/GroupWindowTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/batch/table/GroupWindowTest.scala @@ -449,4 +449,49 @@ class GroupWindowTest extends TableTestBase { util.verifyTable(windowedTable, expected) } + + @Test + def testDecomposableAggFunctions(): Unit = { + val util = batchTestUtil() + val table = util.addTable[(Long, Int, String, Long)]('rowtime, 'a, 'b, 'c) + + val windowedTable = table + .window(Tumble over 15.minutes on 'rowtime as 'w) + .groupBy('w) + .select('c.varPop, 'c.varSamp, 'c.stddevPop, 'c.stddevSamp, 'w.start, 'w.end) + + val expected = + unaryNode( + "DataSetCalc", + unaryNode( + "DataSetWindowAggregate", + unaryNode( + "DataSetCalc", + batchTableNode(0), + term("select", "c", "rowtime", + "*(c, c) AS $f2", "*(c, c) AS $f3", "*(c, c) AS $f4", "*(c, c) AS $f5") + ), + term("window", TumblingGroupWindow('w, 'rowtime, 900000.millis)), + term("select", + "SUM($f2) AS $f0", + "SUM(c) AS $f1", + "COUNT(c) AS $f2", + "SUM($f3) AS $f3", + "SUM($f4) AS $f4", + "SUM($f5) AS $f5", + "start('w) AS TMP_4", + "end('w) AS TMP_5") + ), + term("select", + "CAST(/(-($f0, /(*($f1, $f1), $f2)), $f2)) AS TMP_0", + "CAST(/(-($f3, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1)))) AS TMP_1", + "CAST(POWER(/(-($f4, /(*($f1, $f1), $f2)), $f2), 0.5)) AS TMP_2", + "CAST(POWER(/(-($f5, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1))), 0.5)) " + + "AS TMP_3", + "TMP_4", + "TMP_5") + ) + + util.verifyTable(windowedTable, expected) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/GroupWindowTest.scala index d7d5f1e07b947..d29283456d9b5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/GroupWindowTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/sql/GroupWindowTest.scala @@ -260,4 +260,50 @@ class GroupWindowTest extends TableTestBase { streamUtil.verifySql(sql, expected) } + + @Test + def testDecomposableAggFunctions() = { + + val sql = + "SELECT " + + " VAR_POP(c), VAR_SAMP(c), STDDEV_POP(c), STDDEV_SAMP(c), " + + " TUMBLE_START(rowtime, INTERVAL '15' MINUTE), " + + " TUMBLE_END(rowtime, INTERVAL '15' MINUTE)" + + "FROM MyTable " + + "GROUP BY TUMBLE(rowtime, INTERVAL '15' MINUTE)" + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamGroupWindowAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "rowtime", "c", + "*(c, c) AS $f2", "*(c, c) AS $f3", "*(c, c) AS $f4", "*(c, c) AS $f5") + ), + term("window", TumblingGroupWindow('w$, 'rowtime, 900000.millis)), + term("select", + "SUM($f2) AS $f0", + "SUM(c) AS $f1", + "COUNT(c) AS $f2", + "SUM($f3) AS $f3", + "SUM($f4) AS $f4", + "SUM($f5) AS $f5", + "start('w$) AS w$start", + "end('w$) AS w$end", + "rowtime('w$) AS w$rowtime", + "proctime('w$) AS w$proctime") + ), + term("select", + "CAST(/(-($f0, /(*($f1, $f1), $f2)), $f2)) AS EXPR$0", + "CAST(/(-($f3, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1)))) AS EXPR$1", + "CAST(POWER(/(-($f4, /(*($f1, $f1), $f2)), $f2), 0.5)) AS EXPR$2", + "CAST(POWER(/(-($f5, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1))), 0.5)) " + + "AS EXPR$3", + "w$start AS EXPR$4", + "w$end AS EXPR$5") + ) + streamUtil.verifySql(sql, expected) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/GroupWindowTest.scala index 260726ba495b9..a59ad8382a0ed 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/GroupWindowTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/stream/table/GroupWindowTest.scala @@ -782,4 +782,49 @@ class GroupWindowTest extends TableTestBase { util.verifyTable(windowedTable, expected) } + + @Test + def testDecomposableAggFunctions(): Unit = { + val util = streamTestUtil() + val table = util.addTable[(Long, Int, String, Long)]('rowtime.rowtime, 'a, 'b, 'c) + + val windowedTable = table + .window(Tumble over 15.minutes on 'rowtime as 'w) + .groupBy('w) + .select('c.varPop, 'c.varSamp, 'c.stddevPop, 'c.stddevSamp, 'w.start, 'w.end) + + val expected = + unaryNode( + "DataStreamCalc", + unaryNode( + "DataStreamGroupWindowAggregate", + unaryNode( + "DataStreamCalc", + streamTableNode(0), + term("select", "c", "rowtime", + "*(c, c) AS $f2", "*(c, c) AS $f3", "*(c, c) AS $f4", "*(c, c) AS $f5") + ), + term("window", TumblingGroupWindow('w, 'rowtime, 900000.millis)), + term("select", + "SUM($f2) AS $f0", + "SUM(c) AS $f1", + "COUNT(c) AS $f2", + "SUM($f3) AS $f3", + "SUM($f4) AS $f4", + "SUM($f5) AS $f5", + "start('w) AS TMP_4", + "end('w) AS TMP_5") + ), + term("select", + "CAST(/(-($f0, /(*($f1, $f1), $f2)), $f2)) AS TMP_0", + "CAST(/(-($f3, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1)))) AS TMP_1", + "CAST(POWER(/(-($f4, /(*($f1, $f1), $f2)), $f2), 0.5)) AS TMP_2", + "CAST(POWER(/(-($f5, /(*($f1, $f1), $f2)), CASE(=($f2, 1), null, -($f2, 1))), 0.5)) " + + "AS TMP_3", + "TMP_4", + "TMP_5") + ) + + util.verifyTable(windowedTable, expected) + } } From b9aec0eeebc3a8e0d46a15bae8c09266703e8945 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Mar 2018 15:15:08 +0100 Subject: [PATCH 180/268] [hotfix] [core] Fix checkstyle in 'org.apache.flink.api.common.state' --- .../api/common/state/AggregatingState.java | 10 +++--- .../state/AggregatingStateDescriptor.java | 6 ++-- .../api/common/state/AppendingState.java | 26 ++++++-------- .../api/common/state/BroadcastState.java | 4 +-- .../flink/api/common/state/FoldingState.java | 4 +-- .../common/state/FoldingStateDescriptor.java | 2 +- .../api/common/state/KeyedStateStore.java | 2 +- .../flink/api/common/state/ListState.java | 9 ++--- .../api/common/state/ListStateDescriptor.java | 8 ++--- .../flink/api/common/state/MapState.java | 16 ++++----- .../api/common/state/MapStateDescriptor.java | 12 +++---- .../flink/api/common/state/MergingState.java | 2 +- .../api/common/state/OperatorStateStore.java | 11 ++---- .../flink/api/common/state/ReducingState.java | 4 +-- .../common/state/ReducingStateDescriptor.java | 8 ++--- .../flink/api/common/state/StateBinder.java | 4 +-- .../api/common/state/StateDescriptor.java | 12 +++---- .../flink/api/common/state/ValueState.java | 14 ++++---- .../common/state/ValueStateDescriptor.java | 9 ++--- .../state/AggregatingStateDescriptorTest.java | 8 +++-- .../common/state/ListStateDescriptorTest.java | 23 +++++++------ .../common/state/MapStateDescriptorTest.java | 19 ++++++----- .../state/ReducingStateDescriptorTest.java | 34 ++++++++++--------- .../state/ValueStateDescriptorTest.java | 33 ++++++++++-------- tools/maven/suppressions-core.xml | 4 --- 25 files changed, 144 insertions(+), 140 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingState.java index e69fdb411db04..5c72650412b29 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingState.java @@ -22,22 +22,22 @@ import org.apache.flink.api.common.functions.AggregateFunction; /** - * {@link State} interface for aggregating state, based on an + * {@link State} interface for aggregating state, based on an * {@link AggregateFunction}. Elements that are added to this type of state will * be eagerly pre-aggregated using a given {@code AggregateFunction}. - * + * *

    The state holds internally always the accumulator type of the {@code AggregateFunction}. - * When accessing the result of the state, the function's + * When accessing the result of the state, the function's * {@link AggregateFunction#getResult(Object)} method. * *

    The state is accessed and modified by user functions, and checkpointed consistently * by the system as part of the distributed snapshots. - * + * *

    The state is only accessible by functions applied on a {@code KeyedStream}. The key is * automatically supplied by the system, so the function always sees the value mapped to the * key of the current element. That way, the system can handle stream and state partitioning * consistently together. - * + * * @param Type of the value added to the state. * @param Type of the value extracted from the state. */ diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java index b7378d6a30b24..6f6d2f9790e08 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java @@ -30,7 +30,7 @@ * *

    The type internally stored in the state is the type of the {@code Accumulator} of the * {@code AggregateFunction}. - * + * * @param The type of the values that are added to the state. * @param The type of the accumulator (intermediate aggregation state). * @param The type of the values that are returned from the state. @@ -39,7 +39,7 @@ public class AggregatingStateDescriptor extends StateDescriptor, ACC> { private static final long serialVersionUID = 1L; - /** The aggregation function for the state */ + /** The aggregation function for the state. */ private final AggregateFunction aggFunction; /** @@ -49,7 +49,7 @@ public class AggregatingStateDescriptor extends StateDescriptorThe state is accessed and modified by user functions, and checkpointed consistently * by the system as part of the distributed snapshots. - * + * *

    The state is only accessible by functions applied on a {@code KeyedStream}. The key is * automatically supplied by the system, so the function always sees the value mapped to the * key of the current element. That way, the system can handle stream and state partitioning * consistently together. - * + * * @param Type of the value that can be added to the state. * @param Type of the value that can be retrieved from the state. */ @@ -47,29 +45,27 @@ public interface AppendingState extends State { * depends on the current operator input, as the operator maintains an * independent state for each partition. * - *

    - * NOTE TO IMPLEMENTERS: if the state is empty, then this method - * should return {@code null}. - *

    + *

    NOTE TO IMPLEMENTERS: if the state is empty, then this method + * should return {@code null}. * * @return The operator state value corresponding to the current input or {@code null} * if the state is empty. - * + * * @throws Exception Thrown if the system cannot access the state. */ - OUT get() throws Exception ; + OUT get() throws Exception; /** * Updates the operator state accessible by {@link #get()} by adding the given value * to the list of values. The next time {@link #get()} is called (for the same state * partition) the returned state will represent the updated list. * - * If `null` is passed in, the state value will remain unchanged - * + *

    If null is passed in, the state value will remain unchanged. + * * @param value The new value for the state. - * - * @throws IOException Thrown if the system cannot access the state. + * + * @throws Exception Thrown if the system cannot access the state. */ void add(IN value) throws Exception; - + } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/BroadcastState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/BroadcastState.java index 0cece41a46f2f..fcc8bbf713feb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/BroadcastState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/BroadcastState.java @@ -29,7 +29,7 @@ * *

    CAUTION: the user has to guarantee that all task instances store the same elements in this type of state. * - *

    Each operator instance individually maintains and stores elements in the broadcast state. The fact that the + *

    Each operator instance individually maintains and stores elements in the broadcast state. The fact that the * incoming stream is a broadcast one guarantees that all instances see all the elements. Upon recovery * or re-scaling, the same state is given to each of the instances. To avoid hotspots, each task reads its previous * partition, and if there are more tasks (scale up), then the new instances read from the old instances in a round @@ -80,7 +80,7 @@ public interface BroadcastState extends ReadOnlyBroadcastState { Iterator> iterator() throws Exception; /** - * Returns all the mappings in the state + * Returns all the mappings in the state. * * @return An iterable view of all the key-value pairs in the state. * diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingState.java index df9a0c6aa4bc9..928e62ba305d0 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingState.java @@ -27,12 +27,12 @@ * *

    The state is accessed and modified by user functions, and checkpointed consistently * by the system as part of the distributed snapshots. - * + * *

    The state is only accessible by functions applied on a {@code KeyedStream}. The key is * automatically supplied by the system, so the function always sees the value mapped to the * key of the current element. That way, the system can handle stream and state partitioning * consistently together. - * + * * @param Type of the values folded into the state * @param Type of the value in the state * diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java index 09540477e9941..261d1fe47219d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java @@ -98,7 +98,7 @@ public FoldingStateDescriptor(String name, ACC initialValue, FoldFunction bind(StateBinder stateBinder) throws Exception { return stateBinder.createFoldingState(this); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java b/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java index a1038a84fc98a..e3726b6f68ed8 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/KeyedStateStore.java @@ -281,5 +281,5 @@ public interface KeyedStateStore { * function (function is not part of a KeyedStream). */ @PublicEvolving - MapState getMapState(MapStateDescriptor stateProperties); + MapState getMapState(MapStateDescriptor stateProperties); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java index 74f275b3c64d4..254dc1d614076 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ListState.java @@ -26,22 +26,23 @@ * {@link State} interface for partitioned list state in Operations. * The state is accessed and modified by user functions, and checkpointed consistently * by the system as part of the distributed snapshots. - * + * *

    The state is only accessible by functions applied on a {@code KeyedStream}. The key is * automatically supplied by the system, so the function always sees the value mapped to the * key of the current element. That way, the system can handle stream and state partitioning * consistently together. - * + * * @param Type of values that this list state keeps. */ @PublicEvolving public interface ListState extends MergingState> { + /** * Updates the operator state accessible by {@link #get()} by updating existing values to * to the given list of values. The next time {@link #get()} is called (for the same state * partition) the returned state will represent the updated list. * - * If `null` or an empty list is passed in, the state value will be null + *

    If null or an empty list is passed in, the state value will be null. * * @param values The new values for the state. * @@ -54,7 +55,7 @@ public interface ListState extends MergingState> { * to existing list of values. The next time {@link #get()} is called (for the same state * partition) the returned state will represent the updated list. * - * If `null` or an empty list is passed in, the state value remains unchanged + *

    If null or an empty list is passed in, the state value remains unchanged. * * @param values The new values to be added to the state. * diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ListStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ListStateDescriptor.java index e59d6ee832d57..38e56803330e5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ListStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ListStateDescriptor.java @@ -29,12 +29,12 @@ /** * A {@link StateDescriptor} for {@link ListState}. This can be used to create state where the type * is a list that can be appended and iterated over. - * + * *

    Using {@code ListState} is typically more efficient than manually maintaining a list in a * {@link ValueState}, because the backing implementation can support efficient appends, rather than * replacing the full list on write. - * - *

    To create keyed list state (on a KeyedStream), use + * + *

    To create keyed list state (on a KeyedStream), use * {@link org.apache.flink.api.common.functions.RuntimeContext#getListState(ListStateDescriptor)}. * * @param The type of the values that can be added to the list state. @@ -85,7 +85,7 @@ public ListState bind(StateBinder stateBinder) throws Exception { /** * Gets the serializer for the elements contained in the list. - * + * * @return The serializer for the elements in the list. */ public TypeSerializer getElementSerializer() { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/MapState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/MapState.java index f37fddd67e04a..7a130d49083d2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/MapState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/MapState.java @@ -60,7 +60,7 @@ public interface MapState extends State { * @throws Exception Thrown if the system cannot access the state. */ void put(UK key, UV value) throws Exception; - + /** * Copies all of the mappings from the given map into the state. * @@ -90,19 +90,19 @@ public interface MapState extends State { boolean contains(UK key) throws Exception; /** - * Returns all the mappings in the state + * Returns all the mappings in the state. * * @return An iterable view of all the key-value pairs in the state. - * + * * @throws Exception Thrown if the system cannot access the state. */ Iterable> entries() throws Exception; - + /** - * Returns all the keys in the state + * Returns all the keys in the state. * * @return An iterable view of all the keys in the state. - * + * * @throws Exception Thrown if the system cannot access the state. */ Iterable keys() throws Exception; @@ -111,7 +111,7 @@ public interface MapState extends State { * Returns all the values in the state. * * @return An iterable view of all the values in the state. - * + * * @throws Exception Thrown if the system cannot access the state. */ Iterable values() throws Exception; @@ -120,7 +120,7 @@ public interface MapState extends State { * Iterates over all the mappings in the state. * * @return An iterator over all the mappings in the state - * + * * @throws Exception Thrown if the system cannot access the state. */ Iterator> iterator() throws Exception; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java index 16c00cb439430..2e7ac98778f32 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java @@ -29,12 +29,12 @@ /** * A {@link StateDescriptor} for {@link MapState}. This can be used to create state where the type * is a map that can be updated and iterated over. - * + * *

    Using {@code MapState} is typically more efficient than manually maintaining a map in a * {@link ValueState}, because the backing implementation can support efficient updates, rather then * replacing the full map on write. - * - *

    To create keyed map state (on a KeyedStream), use + * + *

    To create keyed map state (on a KeyedStream), use * {@link org.apache.flink.api.common.functions.RuntimeContext#getMapState(MapStateDescriptor)}. * * @param The type of the keys that can be added to the map state. @@ -90,7 +90,7 @@ public Type getType() { /** * Gets the serializer for the keys in the state. - * + * * @return The serializer for the keys in the state. */ public TypeSerializer getKeySerializer() { @@ -115,7 +115,7 @@ public TypeSerializer getValueSerializer() { return ((MapSerializer) rawSerializer).getValueSerializer(); } - + @Override public int hashCode() { int result = serializer.hashCode(); @@ -128,7 +128,7 @@ public boolean equals(Object o) { if (this == o) { return true; } - + if (o == null || getClass() != o.getClass()) { return false; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/MergingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/MergingState.java index e79f90746252b..8c1631336139c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/MergingState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/MergingState.java @@ -24,7 +24,7 @@ * Extension of {@link AppendingState} that allows merging of state. That is, two instances * of {@link MergingState} can be combined into a single instance that contains all the * information of the two merged states. - * + * * @param Type of the value that can be added to the state. * @param Type of the value that can be retrieved from the state. */ diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorStateStore.java b/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorStateStore.java index c2037e0b58425..7a998e6149c98 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorStateStore.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorStateStore.java @@ -49,8 +49,7 @@ public interface OperatorStateStore { * @param The type of the keys in the broadcast state. * @param The type of the values in the broadcast state. * - * @return The {@link BroadcastState Broadcast State}. - * @throws Exception + * @return The Broadcast State */ BroadcastState getBroadcastState(MapStateDescriptor stateDescriptor) throws Exception; @@ -73,7 +72,6 @@ public interface OperatorStateStore { * @param The generic type of the state * * @return A list for all state partitions. - * @throws Exception */ ListState getListState(ListStateDescriptor stateDescriptor) throws Exception; @@ -97,7 +95,6 @@ public interface OperatorStateStore { * @param The generic type of the state * * @return A list for all state partitions. - * @throws Exception */ ListState getUnionListState(ListStateDescriptor stateDescriptor) throws Exception; @@ -123,13 +120,12 @@ public interface OperatorStateStore { * Creates (or restores) a list state. Each state is registered under a unique name. * The provided serializer is used to de/serialize the state in case of checkpointing (snapshot/restore). * - * The items in the list are repartitionable by the system in case of changed operator parallelism. + *

    The items in the list are repartitionable by the system in case of changed operator parallelism. * * @param stateDescriptor The descriptor for this state, providing a name and serializer. * @param The generic type of the state * * @return A list for all state partitions. - * @throws Exception * * @deprecated since 1.3.0. This was deprecated as part of a refinement to the function names. * Please use {@link #getListState(ListStateDescriptor)} instead. @@ -140,13 +136,12 @@ public interface OperatorStateStore { /** * Creates a state of the given name that uses Java serialization to persist the state. The items in the list * are repartitionable by the system in case of changed operator parallelism. - * + * *

    This is a simple convenience method. For more flexibility on how state serialization * should happen, use the {@link #getListState(ListStateDescriptor)} method. * * @param stateName The name of state to create * @return A list state using Java serialization to serialize state objects. - * @throws Exception * * @deprecated since 1.3.0. Using Java serialization for persisting state is not encouraged. * Please use {@link #getListState(ListStateDescriptor)} instead. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingState.java index 25777ebcc4937..0fe3ed9453b3c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingState.java @@ -26,12 +26,12 @@ * *

    The state is accessed and modified by user functions, and checkpointed consistently * by the system as part of the distributed snapshots. - * + * *

    The state is only accessible by functions applied on a {@code KeyedStream}. The key is * automatically supplied by the system, so the function always sees the value mapped to the * key of the current element. That way, the system can handle stream and state partitioning * consistently together. - * + * * @param Type of the value in the operator state */ @PublicEvolving diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java index 3edf1caf8a9aa..a14b4bd181557 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java @@ -35,9 +35,9 @@ */ @PublicEvolving public class ReducingStateDescriptor extends StateDescriptor, T> { + private static final long serialVersionUID = 1L; - - + private final ReduceFunction reduceFunction; /** @@ -47,7 +47,7 @@ public class ReducingStateDescriptor extends StateDescriptor * consider using the {@link #ReducingStateDescriptor(String, ReduceFunction, TypeInformation)} constructor. * * @param name The (unique) name for the state. - * @param reduceFunction The {@code ReduceFunction} used to aggregate the state. + * @param reduceFunction The {@code ReduceFunction} used to aggregate the state. * @param typeClass The type of the values in the state. */ public ReducingStateDescriptor(String name, ReduceFunction reduceFunction, Class typeClass) { @@ -84,7 +84,7 @@ public ReducingStateDescriptor(String name, ReduceFunction reduceFunction, Ty } // ------------------------------------------------------------------------ - + @Override public ReducingState bind(StateBinder stateBinder) throws Exception { return stateBinder.createReducingState(this); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateBinder.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateBinder.java index a1f7d8d2939f5..871b4a8eb371d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateBinder.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateBinder.java @@ -56,8 +56,8 @@ public interface StateBinder { * @param stateDesc The {@code StateDescriptor} that contains the name of the state. * * @param The type of the values that go into the aggregating state - * @param The type of the values that are stored in the aggregating state - * @param The type of the values that come out of the aggregating state + * @param The type of the values that are stored in the aggregating state + * @param The type of the values that come out of the aggregating state */ AggregatingState createAggregatingState( AggregatingStateDescriptor stateDesc) throws Exception; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java index b603c719ad856..841f710db5917 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java @@ -82,7 +82,7 @@ public enum Type { /** Name for queries against state created from this StateDescriptor. */ private String queryableStateName; - /** The default value returned by the state when no other value is bound to a key */ + /** The default value returned by the state when no other value is bound to a key. */ protected transient T defaultValue; /** The type information describing the value type. Only used to lazily create the serializer @@ -111,7 +111,7 @@ protected StateDescriptor(String name, TypeSerializer serializer, T defaultVa * @param name The name of the {@code StateDescriptor}. * @param typeInfo The type information for the values in the state. * @param defaultValue The default value that will be set when requesting state without setting - * a value before. + * a value before. */ protected StateDescriptor(String name, TypeInformation typeInfo, T defaultValue) { this.name = requireNonNull(name, "name must not be null"); @@ -301,8 +301,8 @@ private void writeObject(final ObjectOutputStream out) throws IOException { byte[] serializedDefaultValue; try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputViewStreamWrapper outView = new DataOutputViewStreamWrapper(baos)) - { + DataOutputViewStreamWrapper outView = new DataOutputViewStreamWrapper(baos)) { + TypeSerializer duplicateSerializer = serializer.duplicate(); duplicateSerializer.serialize(defaultValue, outView); @@ -333,8 +333,8 @@ private void readObject(final ObjectInputStream in) throws IOException, ClassNot in.readFully(buffer); try (ByteArrayInputStream bais = new ByteArrayInputStream(buffer); - DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(bais)) - { + DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(bais)) { + defaultValue = serializer.deserialize(inView); } catch (Exception e) { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ValueState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ValueState.java index ac5571516d970..777e84af0b095 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ValueState.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ValueState.java @@ -28,12 +28,12 @@ * *

    The state is accessed and modified by user functions, and checkpointed consistently * by the system as part of the distributed snapshots. - * + * *

    The state is only accessible by functions applied on a {@code KeyedStream}. The key is * automatically supplied by the system, so the function always sees the value mapped to the * key of the current element. That way, the system can handle stream and state partitioning * consistently together. - * + * * @param Type of the value in the state. */ @PublicEvolving @@ -50,7 +50,7 @@ public interface ValueState extends State { * this will return {@code null} when to value was previously set using {@link #update(Object)}. * * @return The state value corresponding to the current input. - * + * * @throws IOException Thrown if the system cannot access the state. */ T value() throws IOException; @@ -59,13 +59,13 @@ public interface ValueState extends State { * Updates the operator state accessible by {@link #value()} to the given * value. The next time {@link #value()} is called (for the same state * partition) the returned state will represent the updated value. When a - * partitioned state is updated with null, the state for the current key + * partitioned state is updated with null, the state for the current key * will be removed and the default value is returned on the next access. - * + * * @param value The new value for the state. - * + * * @throws IOException Thrown if the system cannot access the state. */ void update(T value) throws IOException; - + } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ValueStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ValueStateDescriptor.java index 3afc8a7b185b7..ef18d741209c3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ValueStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ValueStateDescriptor.java @@ -34,11 +34,12 @@ */ @PublicEvolving public class ValueStateDescriptor extends StateDescriptor, T> { + private static final long serialVersionUID = 1L; - + /** * Creates a new {@code ValueStateDescriptor} with the given name, type, and default value. - * + * *

    If this constructor fails (because it is not possible to describe the type via a class), * consider using the {@link #ValueStateDescriptor(String, TypeInformation, Object)} constructor. * @@ -46,7 +47,7 @@ public class ValueStateDescriptor extends StateDescriptor, T> { * the default value by checking whether the contents of the state is {@code null}. * * @param name The (unique) name for the state. - * @param typeClass The type of the values in the state. + * @param typeClass The type of the values in the state. * @param defaultValue The default value that will be set when requesting state without setting * a value before. */ @@ -122,7 +123,7 @@ public ValueStateDescriptor(String name, TypeSerializer typeSerializer) { } // ------------------------------------------------------------------------ - + @Override public ValueState bind(StateBinder stateBinder) throws Exception { return stateBinder.createValueState(this); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java index 1b27ebd89ee1a..155f23a9c637e 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.util.TestLogger; + import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -29,12 +30,15 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +/** + * Tests for the {@link AggregatingStateDescriptor}. + */ public class AggregatingStateDescriptorTest extends TestLogger { /** - * FLINK-6775 + * FLINK-6775. * - * Tests that the returned serializer is duplicated. This allows to + *

    Tests that the returned serializer is duplicated. This allows to * share the state descriptor. */ @SuppressWarnings("unchecked") diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java index 0b230ad06d06c..c6d086e695c56 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java @@ -41,16 +41,19 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +/** + * Tests for the {@link ListStateDescriptor}. + */ public class ListStateDescriptorTest { - + @Test public void testValueStateDescriptorEagerSerializer() throws Exception { TypeSerializer serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); - - ListStateDescriptor descr = - new ListStateDescriptor("testName", serializer); - + + ListStateDescriptor descr = + new ListStateDescriptor<>("testName", serializer); + assertEquals("testName", descr.getName()); assertNotNull(descr.getSerializer()); assertTrue(descr.getSerializer() instanceof ListSerializer); @@ -74,8 +77,8 @@ public void testValueStateDescriptorLazySerializer() throws Exception { cfg.registerKryoType(TaskInfo.class); ListStateDescriptor descr = - new ListStateDescriptor("testName", Path.class); - + new ListStateDescriptor<>("testName", Path.class); + try { descr.getSerializer(); fail("should cause an exception"); @@ -96,7 +99,7 @@ public void testValueStateDescriptorLazySerializer() throws Exception { public void testValueStateDescriptorAutoSerializer() throws Exception { ListStateDescriptor descr = - new ListStateDescriptor("testName", String.class); + new ListStateDescriptor<>("testName", String.class); ListStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); @@ -110,9 +113,9 @@ public void testValueStateDescriptorAutoSerializer() throws Exception { } /** - * FLINK-6775 + * FLINK-6775. * - * Tests that the returned serializer is duplicated. This allows to + *

    Tests that the returned serializer is duplicated. This allows to * share the state descriptor. */ @SuppressWarnings("unchecked") diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java index d71091141d7a1..e2aa940351d2a 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java @@ -42,17 +42,20 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +/** + * Tests for the {@link MapStateDescriptor}. + */ public class MapStateDescriptorTest { - + @Test public void testMapStateDescriptorEagerSerializer() throws Exception { TypeSerializer keySerializer = new KryoSerializer<>(Integer.class, new ExecutionConfig()); TypeSerializer valueSerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); - - MapStateDescriptor descr = + + MapStateDescriptor descr = new MapStateDescriptor<>("testName", keySerializer, valueSerializer); - + assertEquals("testName", descr.getName()); assertNotNull(descr.getSerializer()); assertTrue(descr.getSerializer() instanceof MapSerializer); @@ -81,7 +84,7 @@ public void testMapStateDescriptorLazySerializer() throws Exception { MapStateDescriptor descr = new MapStateDescriptor<>("testName", Path.class, String.class); - + try { descr.getSerializer(); fail("should cause an exception"); @@ -96,7 +99,7 @@ public void testMapStateDescriptorLazySerializer() throws Exception { assertTrue(descr.getKeySerializer() instanceof KryoSerializer); assertTrue(((KryoSerializer) descr.getKeySerializer()).getKryo().getRegistration(TaskInfo.class).getId() > 0); - + assertNotNull(descr.getValueSerializer()); assertTrue(descr.getValueSerializer() instanceof StringSerializer); } @@ -121,9 +124,9 @@ public void testMapStateDescriptorAutoSerializer() throws Exception { } /** - * FLINK-6775 + * FLINK-6775. * - * Tests that the returned serializer is duplicated. This allows to + *

    Tests that the returned serializer is duplicated. This allows to * share the state descriptor. */ @SuppressWarnings("unchecked") diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java index aec7140226564..ef39f1496c722 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java @@ -26,8 +26,8 @@ import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CommonTestUtils; - import org.apache.flink.util.TestLogger; + import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -36,24 +36,26 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; - import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +/** + * Tests for the {@link ReducingStateDescriptor}. + */ public class ReducingStateDescriptorTest extends TestLogger { - + @Test public void testValueStateDescriptorEagerSerializer() throws Exception { @SuppressWarnings("unchecked") - ReduceFunction reducer = mock(ReduceFunction.class); - + ReduceFunction reducer = mock(ReduceFunction.class); + TypeSerializer serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); - - ReducingStateDescriptor descr = - new ReducingStateDescriptor("testName", reducer, serializer); - + + ReducingStateDescriptor descr = + new ReducingStateDescriptor<>("testName", reducer, serializer); + assertEquals("testName", descr.getName()); assertNotNull(descr.getSerializer()); assertEquals(serializer, descr.getSerializer()); @@ -70,13 +72,13 @@ public void testValueStateDescriptorLazySerializer() throws Exception { @SuppressWarnings("unchecked") ReduceFunction reducer = mock(ReduceFunction.class); - + // some different registered value ExecutionConfig cfg = new ExecutionConfig(); cfg.registerKryoType(TaskInfo.class); ReducingStateDescriptor descr = - new ReducingStateDescriptor("testName", reducer, Path.class); + new ReducingStateDescriptor<>("testName", reducer, Path.class); try { descr.getSerializer(); @@ -84,7 +86,7 @@ public void testValueStateDescriptorLazySerializer() throws Exception { } catch (IllegalStateException ignored) {} descr.initializeSerializerUnlessSet(cfg); - + assertNotNull(descr.getSerializer()); assertTrue(descr.getSerializer() instanceof KryoSerializer); @@ -98,7 +100,7 @@ public void testValueStateDescriptorAutoSerializer() throws Exception { ReduceFunction reducer = mock(ReduceFunction.class); ReducingStateDescriptor descr = - new ReducingStateDescriptor("testName", reducer, String.class); + new ReducingStateDescriptor<>("testName", reducer, String.class); ReducingStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); @@ -108,9 +110,9 @@ public void testValueStateDescriptorAutoSerializer() throws Exception { } /** - * FLINK-6775 + * FLINK-6775. * - * Tests that the returned serializer is duplicated. This allows to + *

    Tests that the returned serializer is duplicated. This allows to * share the state descriptor. */ @SuppressWarnings("unchecked") @@ -134,5 +136,5 @@ public TypeSerializer answer(InvocationOnMock invocation) throws Throwab // check that the retrieved serializers are not the same assertNotSame(serializerA, serializerB); } - + } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java index e434e011da9e3..b43e5ad16345f 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java @@ -26,8 +26,8 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CommonTestUtils; - import org.apache.flink.util.TestLogger; + import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -42,17 +42,20 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +/** + * Tests for the {@link ValueStateDescriptor}. + */ public class ValueStateDescriptorTest extends TestLogger { - + @Test public void testValueStateDescriptorEagerSerializer() throws Exception { TypeSerializer serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); String defaultValue = "le-value-default"; - - ValueStateDescriptor descr = - new ValueStateDescriptor("testName", serializer, defaultValue); - + + ValueStateDescriptor descr = + new ValueStateDescriptor<>("testName", serializer, defaultValue); + assertEquals("testName", descr.getName()); assertEquals(defaultValue, descr.getDefaultValue()); assertNotNull(descr.getSerializer()); @@ -68,16 +71,16 @@ public void testValueStateDescriptorEagerSerializer() throws Exception { @Test public void testValueStateDescriptorLazySerializer() throws Exception { - + // some default value that goes to the generic serializer Path defaultValue = new Path(new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).toURI()); - + // some different registered value ExecutionConfig cfg = new ExecutionConfig(); cfg.registerKryoType(TaskInfo.class); ValueStateDescriptor descr = - new ValueStateDescriptor("testName", Path.class, defaultValue); + new ValueStateDescriptor<>("testName", Path.class, defaultValue); try { descr.getSerializer(); @@ -85,7 +88,7 @@ public void testValueStateDescriptorLazySerializer() throws Exception { } catch (IllegalStateException ignored) {} descr.initializeSerializerUnlessSet(cfg); - + assertNotNull(descr.getSerializer()); assertTrue(descr.getSerializer() instanceof KryoSerializer); @@ -94,11 +97,11 @@ public void testValueStateDescriptorLazySerializer() throws Exception { @Test public void testValueStateDescriptorAutoSerializer() throws Exception { - + String defaultValue = "le-value-default"; ValueStateDescriptor descr = - new ValueStateDescriptor("testName", String.class, defaultValue); + new ValueStateDescriptor<>("testName", String.class, defaultValue); ValueStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); @@ -122,7 +125,7 @@ public void testVeryLargeDefaultValue() throws Exception { String defaultValue = new String(data, ConfigConstants.DEFAULT_CHARSET); ValueStateDescriptor descr = - new ValueStateDescriptor("testName", serializer, defaultValue); + new ValueStateDescriptor<>("testName", serializer, defaultValue); assertEquals("testName", descr.getName()); assertEquals(defaultValue, descr.getDefaultValue()); @@ -138,9 +141,9 @@ public void testVeryLargeDefaultValue() throws Exception { } /** - * FLINK-6775 + * FLINK-6775. * - * Tests that the returned serializer is duplicated. This allows to + *

    Tests that the returned serializer is duplicated. This allows to * share the state descriptor. */ @SuppressWarnings("unchecked") diff --git a/tools/maven/suppressions-core.xml b/tools/maven/suppressions-core.xml index e613fb0cfe3cc..ff9c2038583d0 100644 --- a/tools/maven/suppressions-core.xml +++ b/tools/maven/suppressions-core.xml @@ -71,10 +71,6 @@ under the License. files="(.*)test[/\\](.*)api[/\\]common[/\\]operators[/\\](.*)" checks="AvoidStarImport"/> - - From 172af4c96eb5ab6a69ca71471551f42da548d713 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Mar 2018 15:29:12 +0100 Subject: [PATCH 181/268] [hotfix] [core] Add missing serialVersionUID to MapStateDescriptor --- .../org/apache/flink/api/common/state/MapStateDescriptor.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java index 2e7ac98778f32..087cb5410bc59 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java @@ -42,6 +42,8 @@ @PublicEvolving public class MapStateDescriptor extends StateDescriptor, Map> { + private static final long serialVersionUID = 1L; + /** * Create a new {@code MapStateDescriptor} with the given name and the given type serializers. * From 9e57a1d220fed6bb8d63166e77ae8a5619e37878 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Mar 2018 15:36:19 +0100 Subject: [PATCH 182/268] [hotfix] [core] Demockitofy state descriptor tests --- .../state/AggregatingStateDescriptorTest.java | 16 ++++--------- .../common/state/ListStateDescriptorTest.java | 14 +++-------- .../common/state/MapStateDescriptorTest.java | 23 ++++--------------- .../state/ReducingStateDescriptorTest.java | 22 +++++++----------- .../state/ValueStateDescriptorTest.java | 14 +++-------- 5 files changed, 23 insertions(+), 66 deletions(-) diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java index 155f23a9c637e..f62acc8996fd5 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java @@ -18,17 +18,16 @@ package org.apache.flink.api.common.state; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.util.TestLogger; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import static org.junit.Assert.assertNotSame; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Tests for the {@link AggregatingStateDescriptor}. @@ -41,16 +40,11 @@ public class AggregatingStateDescriptorTest extends TestLogger { *

    Tests that the returned serializer is duplicated. This allows to * share the state descriptor. */ - @SuppressWarnings("unchecked") @Test public void testSerializerDuplication() { - TypeSerializer serializer = mock(TypeSerializer.class); - when(serializer.duplicate()).thenAnswer(new Answer>() { - @Override - public TypeSerializer answer(InvocationOnMock invocation) throws Throwable { - return mock(TypeSerializer.class); - } - }); + // we need a serializer that actually duplicates for testing (a stateful one) + // we use Kryo here, because it meets these conditions + TypeSerializer serializer = new KryoSerializer<>(Long.class, new ExecutionConfig()); AggregateFunction aggregatingFunction = mock(AggregateFunction.class); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java index c6d086e695c56..f45d2965ea1ec 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java @@ -28,8 +28,6 @@ import org.apache.flink.core.testutils.CommonTestUtils; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import java.util.List; @@ -38,8 +36,6 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Tests for the {@link ListStateDescriptor}. @@ -121,13 +117,9 @@ public void testValueStateDescriptorAutoSerializer() throws Exception { @SuppressWarnings("unchecked") @Test public void testSerializerDuplication() { - TypeSerializer statefulSerializer = mock(TypeSerializer.class); - when(statefulSerializer.duplicate()).thenAnswer(new Answer>() { - @Override - public TypeSerializer answer(InvocationOnMock invocation) throws Throwable { - return mock(TypeSerializer.class); - } - }); + // we need a serializer that actually duplicates for testing (a stateful one) + // we use Kryo here, because it meets these conditions + TypeSerializer statefulSerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); ListStateDescriptor descr = new ListStateDescriptor<>("foobar", statefulSerializer); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java index e2aa940351d2a..21518347d69b0 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java @@ -29,8 +29,6 @@ import org.apache.flink.core.testutils.CommonTestUtils; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import java.util.Map; @@ -39,8 +37,6 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Tests for the {@link MapStateDescriptor}. @@ -129,23 +125,12 @@ public void testMapStateDescriptorAutoSerializer() throws Exception { *

    Tests that the returned serializer is duplicated. This allows to * share the state descriptor. */ - @SuppressWarnings("unchecked") @Test public void testSerializerDuplication() { - TypeSerializer keySerializer = mock(TypeSerializer.class); - TypeSerializer valueSerializer = mock(TypeSerializer.class); - when(keySerializer.duplicate()).thenAnswer(new Answer>() { - @Override - public TypeSerializer answer(InvocationOnMock invocation) throws Throwable { - return mock(TypeSerializer.class); - } - }); - when(valueSerializer.duplicate()).thenAnswer(new Answer>() { - @Override - public TypeSerializer answer(InvocationOnMock invocation) throws Throwable { - return mock(TypeSerializer.class); - } - }); + // we need a serializer that actually duplicates for testing (a stateful one) + // we use Kryo here, because it meets these conditions + TypeSerializer keySerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); + TypeSerializer valueSerializer = new KryoSerializer<>(Long.class, new ExecutionConfig()); MapStateDescriptor descr = new MapStateDescriptor<>("foobar", keySerializer, valueSerializer); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java index ef39f1496c722..1e21a78d19da8 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java @@ -29,8 +29,6 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -38,7 +36,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Tests for the {@link ReducingStateDescriptor}. @@ -118,17 +115,14 @@ public void testValueStateDescriptorAutoSerializer() throws Exception { @SuppressWarnings("unchecked") @Test public void testSerializerDuplication() { - TypeSerializer statefulSerializer = mock(TypeSerializer.class); - when(statefulSerializer.duplicate()).thenAnswer(new Answer>() { - @Override - public TypeSerializer answer(InvocationOnMock invocation) throws Throwable { - return mock(TypeSerializer.class); - } - }); - - ReduceFunction reducer = mock(ReduceFunction.class); - - ReducingStateDescriptor descr = new ReducingStateDescriptor<>("foobar", reducer, statefulSerializer); + // we need a serializer that actually duplicates for testing (a stateful one) + // we use Kryo here, because it meets these conditions + TypeSerializer statefulSerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); + + ReducingStateDescriptor descr = new ReducingStateDescriptor<>( + "foobar", + (a, b) -> a, + statefulSerializer); TypeSerializer serializerA = descr.getSerializer(); TypeSerializer serializerB = descr.getSerializer(); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java index b43e5ad16345f..f3b9eee93f225 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java @@ -29,8 +29,6 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import java.io.File; @@ -39,8 +37,6 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; /** * Tests for the {@link ValueStateDescriptor}. @@ -149,13 +145,9 @@ public void testVeryLargeDefaultValue() throws Exception { @SuppressWarnings("unchecked") @Test public void testSerializerDuplication() { - TypeSerializer statefulSerializer = mock(TypeSerializer.class); - when(statefulSerializer.duplicate()).thenAnswer(new Answer>() { - @Override - public TypeSerializer answer(InvocationOnMock invocation) throws Throwable { - return mock(TypeSerializer.class); - } - }); + // we need a serializer that actually duplicates for testing (a stateful one) + // we use Kryo here, because it meets these conditions + TypeSerializer statefulSerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); ValueStateDescriptor descr = new ValueStateDescriptor<>("foobar", statefulSerializer); From 72c96157ed18d9203a34de5228e1e57708b3aab6 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Mar 2018 15:44:27 +0100 Subject: [PATCH 183/268] [hotfix] [core] Make State Descriptors consistently use Preconditions instead of Objects. --- .../api/common/state/ReducingStateDescriptor.java | 8 ++++---- .../flink/api/common/state/StateDescriptor.java | 14 +++++++------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java index a14b4bd181557..ef483e2af0422 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java @@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import static java.util.Objects.requireNonNull; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * {@link StateDescriptor} for {@link ReducingState}. This can be used to create partitioned @@ -52,7 +52,7 @@ public class ReducingStateDescriptor extends StateDescriptor */ public ReducingStateDescriptor(String name, ReduceFunction reduceFunction, Class typeClass) { super(name, typeClass, null); - this.reduceFunction = requireNonNull(reduceFunction); + this.reduceFunction = checkNotNull(reduceFunction); if (reduceFunction instanceof RichFunction) { throw new UnsupportedOperationException("ReduceFunction of ReducingState can not be a RichFunction."); @@ -68,7 +68,7 @@ public ReducingStateDescriptor(String name, ReduceFunction reduceFunction, Cl */ public ReducingStateDescriptor(String name, ReduceFunction reduceFunction, TypeInformation typeInfo) { super(name, typeInfo, null); - this.reduceFunction = requireNonNull(reduceFunction); + this.reduceFunction = checkNotNull(reduceFunction); } /** @@ -80,7 +80,7 @@ public ReducingStateDescriptor(String name, ReduceFunction reduceFunction, Ty */ public ReducingStateDescriptor(String name, ReduceFunction reduceFunction, TypeSerializer typeSerializer) { super(name, typeSerializer, null); - this.reduceFunction = requireNonNull(reduceFunction); + this.reduceFunction = checkNotNull(reduceFunction); } // ------------------------------------------------------------------------ diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java index 841f710db5917..5ec59e426c640 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java @@ -34,7 +34,7 @@ import java.io.ObjectOutputStream; import java.io.Serializable; -import static java.util.Objects.requireNonNull; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * Base class for state descriptors. A {@code StateDescriptor} is used for creating partitioned @@ -100,8 +100,8 @@ public enum Type { * a value before. */ protected StateDescriptor(String name, TypeSerializer serializer, T defaultValue) { - this.name = requireNonNull(name, "name must not be null"); - this.serializer = requireNonNull(serializer, "serializer must not be null"); + this.name = checkNotNull(name, "name must not be null"); + this.serializer = checkNotNull(serializer, "serializer must not be null"); this.defaultValue = defaultValue; } @@ -114,8 +114,8 @@ protected StateDescriptor(String name, TypeSerializer serializer, T defaultVa * a value before. */ protected StateDescriptor(String name, TypeInformation typeInfo, T defaultValue) { - this.name = requireNonNull(name, "name must not be null"); - this.typeInfo = requireNonNull(typeInfo, "type information must not be null"); + this.name = checkNotNull(name, "name must not be null"); + this.typeInfo = checkNotNull(typeInfo, "type information must not be null"); this.defaultValue = defaultValue; } @@ -131,8 +131,8 @@ protected StateDescriptor(String name, TypeInformation typeInfo, T defaultVal * a value before. */ protected StateDescriptor(String name, Class type, T defaultValue) { - this.name = requireNonNull(name, "name must not be null"); - requireNonNull(type, "type class must not be null"); + this.name = checkNotNull(name, "name must not be null"); + checkNotNull(type, "type class must not be null"); try { this.typeInfo = TypeExtractor.createTypeInfo(type); From 586eb10084f56524f0bf14fb90c16e696b6a941f Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Mar 2018 16:22:12 +0100 Subject: [PATCH 184/268] [FLINK-9034] [core] StateDescriptor does not throw away TypeInformation upon serialization. Throwing away TypeInformation upon serialization was previously done because the type information was not serializable. Now that it is serializable, we can (and should) keep it to provide consistent user experience, where all serializers respect the ExecutionConfig. --- .../api/common/state/StateDescriptor.java | 41 +++-- .../common/state/ListStateDescriptorTest.java | 48 +---- .../common/state/MapStateDescriptorTest.java | 54 +----- .../state/ReducingStateDescriptorTest.java | 54 +----- .../api/common/state/StateDescriptorTest.java | 171 ++++++++++++++++++ .../state/ValueStateDescriptorTest.java | 71 -------- 6 files changed, 200 insertions(+), 239 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java index 5ec59e426c640..574c83603ec16 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java @@ -27,6 +27,8 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.util.Preconditions; +import javax.annotation.Nullable; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -35,6 +37,7 @@ import java.io.Serializable; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; /** * Base class for state descriptors. A {@code StateDescriptor} is used for creating partitioned @@ -76,19 +79,24 @@ public enum Type { protected final String name; /** The serializer for the type. May be eagerly initialized in the constructor, - * or lazily once the type is serialized or an ExecutionConfig is provided. */ + * or lazily once the {@link #initializeSerializerUnlessSet(ExecutionConfig)} method + * is called. */ + @Nullable protected TypeSerializer serializer; + /** The type information describing the value type. Only used to if the serializer + * is created lazily. */ + @Nullable + private TypeInformation typeInfo; + /** Name for queries against state created from this StateDescriptor. */ + @Nullable private String queryableStateName; /** The default value returned by the state when no other value is bound to a key. */ + @Nullable protected transient T defaultValue; - /** The type information describing the value type. Only used to lazily create the serializer - * and dropped during serialization */ - private transient TypeInformation typeInfo; - // ------------------------------------------------------------------------ /** @@ -99,7 +107,7 @@ public enum Type { * @param defaultValue The default value that will be set when requesting state without setting * a value before. */ - protected StateDescriptor(String name, TypeSerializer serializer, T defaultValue) { + protected StateDescriptor(String name, TypeSerializer serializer, @Nullable T defaultValue) { this.name = checkNotNull(name, "name must not be null"); this.serializer = checkNotNull(serializer, "serializer must not be null"); this.defaultValue = defaultValue; @@ -113,7 +121,7 @@ protected StateDescriptor(String name, TypeSerializer serializer, T defaultVa * @param defaultValue The default value that will be set when requesting state without setting * a value before. */ - protected StateDescriptor(String name, TypeInformation typeInfo, T defaultValue) { + protected StateDescriptor(String name, TypeInformation typeInfo, @Nullable T defaultValue) { this.name = checkNotNull(name, "name must not be null"); this.typeInfo = checkNotNull(typeInfo, "type information must not be null"); this.defaultValue = defaultValue; @@ -130,7 +138,7 @@ protected StateDescriptor(String name, TypeInformation typeInfo, T defaultVal * @param defaultValue The default value that will be set when requesting state without setting * a value before. */ - protected StateDescriptor(String name, Class type, T defaultValue) { + protected StateDescriptor(String name, Class type, @Nullable T defaultValue) { this.name = checkNotNull(name, "name must not be null"); checkNotNull(type, "type class must not be null"); @@ -208,6 +216,7 @@ public void setQueryable(String queryableStateName) { * * @return Queryable state name or null if not set. */ + @Nullable public String getQueryableStateName() { return queryableStateName; } @@ -249,12 +258,13 @@ public boolean isSerializerInitialized() { */ public void initializeSerializerUnlessSet(ExecutionConfig executionConfig) { if (serializer == null) { - if (typeInfo != null) { - serializer = typeInfo.createSerializer(executionConfig); - } else { - throw new IllegalStateException( - "Cannot initialize serializer after TypeInformation was dropped during serialization"); - } + checkState(typeInfo != null, "no serializer and no type info"); + + // instantiate the serializer + serializer = typeInfo.createSerializer(executionConfig); + + // we can drop the type info now, no longer needed + typeInfo = null; } } @@ -285,9 +295,6 @@ public String toString() { // ------------------------------------------------------------------------ private void writeObject(final ObjectOutputStream out) throws IOException { - // make sure we have a serializer before the type information gets lost - initializeSerializerUnlessSet(new ExecutionConfig()); - // write all the non-transient fields out.defaultWriteObject(); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java index f45d2965ea1ec..e7e33e79ca578 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java @@ -19,12 +19,9 @@ package org.apache.flink.api.common.state; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.ListSerializer; -import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CommonTestUtils; import org.junit.Test; @@ -35,7 +32,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * Tests for the {@link ListStateDescriptor}. @@ -43,7 +39,7 @@ public class ListStateDescriptorTest { @Test - public void testValueStateDescriptorEagerSerializer() throws Exception { + public void testListStateDescriptor() throws Exception { TypeSerializer serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); @@ -66,48 +62,6 @@ public void testValueStateDescriptorEagerSerializer() throws Exception { assertEquals(serializer, copy.getElementSerializer()); } - @Test - public void testValueStateDescriptorLazySerializer() throws Exception { - // some different registered value - ExecutionConfig cfg = new ExecutionConfig(); - cfg.registerKryoType(TaskInfo.class); - - ListStateDescriptor descr = - new ListStateDescriptor<>("testName", Path.class); - - try { - descr.getSerializer(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) {} - - descr.initializeSerializerUnlessSet(cfg); - - assertNotNull(descr.getSerializer()); - assertTrue(descr.getSerializer() instanceof ListSerializer); - - assertNotNull(descr.getElementSerializer()); - assertTrue(descr.getElementSerializer() instanceof KryoSerializer); - - assertTrue(((KryoSerializer) descr.getElementSerializer()).getKryo().getRegistration(TaskInfo.class).getId() > 0); - } - - @Test - public void testValueStateDescriptorAutoSerializer() throws Exception { - - ListStateDescriptor descr = - new ListStateDescriptor<>("testName", String.class); - - ListStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); - - assertEquals("testName", copy.getName()); - - assertNotNull(copy.getSerializer()); - assertTrue(copy.getSerializer() instanceof ListSerializer); - - assertNotNull(copy.getElementSerializer()); - assertEquals(StringSerializer.INSTANCE, copy.getElementSerializer()); - } - /** * FLINK-6775. * diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java index 21518347d69b0..4e64c0f436a01 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java @@ -19,13 +19,9 @@ package org.apache.flink.api.common.state; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CommonTestUtils; import org.junit.Test; @@ -36,7 +32,6 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * Tests for the {@link MapStateDescriptor}. @@ -44,7 +39,7 @@ public class MapStateDescriptorTest { @Test - public void testMapStateDescriptorEagerSerializer() throws Exception { + public void testMapStateDescriptor() throws Exception { TypeSerializer keySerializer = new KryoSerializer<>(Integer.class, new ExecutionConfig()); TypeSerializer valueSerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); @@ -72,53 +67,6 @@ public void testMapStateDescriptorEagerSerializer() throws Exception { assertEquals(valueSerializer, copy.getValueSerializer()); } - @Test - public void testMapStateDescriptorLazySerializer() throws Exception { - // some different registered value - ExecutionConfig cfg = new ExecutionConfig(); - cfg.registerKryoType(TaskInfo.class); - - MapStateDescriptor descr = - new MapStateDescriptor<>("testName", Path.class, String.class); - - try { - descr.getSerializer(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) {} - - descr.initializeSerializerUnlessSet(cfg); - - assertNotNull(descr.getSerializer()); - assertTrue(descr.getSerializer() instanceof MapSerializer); - - assertNotNull(descr.getKeySerializer()); - assertTrue(descr.getKeySerializer() instanceof KryoSerializer); - - assertTrue(((KryoSerializer) descr.getKeySerializer()).getKryo().getRegistration(TaskInfo.class).getId() > 0); - - assertNotNull(descr.getValueSerializer()); - assertTrue(descr.getValueSerializer() instanceof StringSerializer); - } - - @Test - public void testMapStateDescriptorAutoSerializer() throws Exception { - - MapStateDescriptor descr = - new MapStateDescriptor<>("testName", String.class, Long.class); - - MapStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); - - assertEquals("testName", copy.getName()); - - assertNotNull(copy.getSerializer()); - assertTrue(copy.getSerializer() instanceof MapSerializer); - - assertNotNull(copy.getKeySerializer()); - assertEquals(StringSerializer.INSTANCE, copy.getKeySerializer()); - assertNotNull(copy.getValueSerializer()); - assertEquals(LongSerializer.INSTANCE, copy.getValueSerializer()); - } - /** * FLINK-6775. * diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java index 1e21a78d19da8..81b7c38cf7965 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java @@ -19,12 +19,9 @@ package org.apache.flink.api.common.state; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.util.TestLogger; @@ -33,9 +30,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Mockito.mock; /** * Tests for the {@link ReducingStateDescriptor}. @@ -43,10 +37,9 @@ public class ReducingStateDescriptorTest extends TestLogger { @Test - public void testValueStateDescriptorEagerSerializer() throws Exception { + public void testReducingStateDescriptor() throws Exception { - @SuppressWarnings("unchecked") - ReduceFunction reducer = mock(ReduceFunction.class); + ReduceFunction reducer = (a, b) -> a; TypeSerializer serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); @@ -56,6 +49,7 @@ public void testValueStateDescriptorEagerSerializer() throws Exception { assertEquals("testName", descr.getName()); assertNotNull(descr.getSerializer()); assertEquals(serializer, descr.getSerializer()); + assertEquals(reducer, descr.getReduceFunction()); ReducingStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); @@ -64,48 +58,6 @@ public void testValueStateDescriptorEagerSerializer() throws Exception { assertEquals(serializer, copy.getSerializer()); } - @Test - public void testValueStateDescriptorLazySerializer() throws Exception { - - @SuppressWarnings("unchecked") - ReduceFunction reducer = mock(ReduceFunction.class); - - // some different registered value - ExecutionConfig cfg = new ExecutionConfig(); - cfg.registerKryoType(TaskInfo.class); - - ReducingStateDescriptor descr = - new ReducingStateDescriptor<>("testName", reducer, Path.class); - - try { - descr.getSerializer(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) {} - - descr.initializeSerializerUnlessSet(cfg); - - assertNotNull(descr.getSerializer()); - assertTrue(descr.getSerializer() instanceof KryoSerializer); - - assertTrue(((KryoSerializer) descr.getSerializer()).getKryo().getRegistration(TaskInfo.class).getId() > 0); - } - - @Test - public void testValueStateDescriptorAutoSerializer() throws Exception { - - @SuppressWarnings("unchecked") - ReduceFunction reducer = mock(ReduceFunction.class); - - ReducingStateDescriptor descr = - new ReducingStateDescriptor<>("testName", reducer, String.class); - - ReducingStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); - - assertEquals("testName", copy.getName()); - assertNotNull(copy.getSerializer()); - assertEquals(StringSerializer.INSTANCE, copy.getSerializer()); - } - /** * FLINK-6775. * diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java new file mode 100644 index 0000000000000..59293f410605f --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java @@ -0,0 +1,171 @@ +/* + * 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.api.common.state; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.testutils.CommonTestUtils; + +import org.junit.Test; + +import java.io.File; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests for the common/shared functionality of {@link StateDescriptor}. + */ +public class StateDescriptorTest { + + @Test + public void testInitializeWithSerializer() throws Exception { + final TypeSerializer serializer = StringSerializer.INSTANCE; + final TestStateDescriptor descr = new TestStateDescriptor<>("test", serializer); + + assertTrue(descr.isSerializerInitialized()); + assertNotNull(descr.getSerializer()); + assertTrue(descr.getSerializer() instanceof StringSerializer); + + // this should not have any effect + descr.initializeSerializerUnlessSet(new ExecutionConfig()); + assertTrue(descr.isSerializerInitialized()); + assertNotNull(descr.getSerializer()); + assertTrue(descr.getSerializer() instanceof StringSerializer); + + TestStateDescriptor clone = CommonTestUtils.createCopySerializable(descr); + assertTrue(clone.isSerializerInitialized()); + assertNotNull(clone.getSerializer()); + assertTrue(clone.getSerializer() instanceof StringSerializer); + } + + @Test + public void testInitializeSerializerBeforeSerialization() throws Exception { + final TestStateDescriptor descr = new TestStateDescriptor<>("test", String.class); + + assertFalse(descr.isSerializerInitialized()); + try { + descr.getSerializer(); + fail("should fail with an exception"); + } catch (IllegalStateException ignored) {} + + descr.initializeSerializerUnlessSet(new ExecutionConfig()); + + assertTrue(descr.isSerializerInitialized()); + assertNotNull(descr.getSerializer()); + assertTrue(descr.getSerializer() instanceof StringSerializer); + + TestStateDescriptor clone = CommonTestUtils.createCopySerializable(descr); + + assertTrue(clone.isSerializerInitialized()); + assertNotNull(clone.getSerializer()); + assertTrue(clone.getSerializer() instanceof StringSerializer); + } + + @Test + public void testInitializeSerializerAfterSerialization() throws Exception { + final TestStateDescriptor descr = new TestStateDescriptor<>("test", String.class); + + assertFalse(descr.isSerializerInitialized()); + try { + descr.getSerializer(); + fail("should fail with an exception"); + } catch (IllegalStateException ignored) {} + + TestStateDescriptor clone = CommonTestUtils.createCopySerializable(descr); + + assertFalse(clone.isSerializerInitialized()); + try { + clone.getSerializer(); + fail("should fail with an exception"); + } catch (IllegalStateException ignored) {} + + clone.initializeSerializerUnlessSet(new ExecutionConfig()); + + assertTrue(clone.isSerializerInitialized()); + assertNotNull(clone.getSerializer()); + assertTrue(clone.getSerializer() instanceof StringSerializer); + } + + @Test + public void testInitializeSerializerAfterSerializationWithCustomConfig() throws Exception { + // guard our test assumptions. + assertEquals("broken test assumption", -1, + new KryoSerializer<>(String.class, new ExecutionConfig()).getKryo() + .getRegistration(File.class).getId()); + + final ExecutionConfig config = new ExecutionConfig(); + config.registerKryoType(File.class); + + final TestStateDescriptor original = new TestStateDescriptor<>("test", Path.class); + TestStateDescriptor clone = CommonTestUtils.createCopySerializable(original); + + clone.initializeSerializerUnlessSet(config); + + // serialized one (later initialized) carries the registration + assertTrue(((KryoSerializer) clone.getSerializer()).getKryo() + .getRegistration(File.class).getId() > 0); + } + + // ------------------------------------------------------------------------ + + private static class TestStateDescriptor extends StateDescriptor { + + private static final long serialVersionUID = 1L; + + TestStateDescriptor(String name, TypeSerializer serializer) { + super(name, serializer, null); + } + + TestStateDescriptor(String name, TypeInformation typeInfo) { + super(name, typeInfo, null); + } + + TestStateDescriptor(String name, Class type) { + super(name, type, null); + } + + @Override + public State bind(StateBinder stateBinder) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public Type getType() { + throw new UnsupportedOperationException(); + } + + @Override + public int hashCode() { + return 584523; + } + + @Override + public boolean equals(Object o) { + return o != null && o.getClass() == TestStateDescriptor.class; + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java index f3b9eee93f225..7ee58fedb9864 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java @@ -19,94 +19,23 @@ package org.apache.flink.api.common.state; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.util.TestLogger; import org.junit.Test; -import java.io.File; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * Tests for the {@link ValueStateDescriptor}. */ public class ValueStateDescriptorTest extends TestLogger { - @Test - public void testValueStateDescriptorEagerSerializer() throws Exception { - - TypeSerializer serializer = new KryoSerializer<>(String.class, new ExecutionConfig()); - String defaultValue = "le-value-default"; - - ValueStateDescriptor descr = - new ValueStateDescriptor<>("testName", serializer, defaultValue); - - assertEquals("testName", descr.getName()); - assertEquals(defaultValue, descr.getDefaultValue()); - assertNotNull(descr.getSerializer()); - assertEquals(serializer, descr.getSerializer()); - - ValueStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); - - assertEquals("testName", copy.getName()); - assertEquals(defaultValue, copy.getDefaultValue()); - assertNotNull(copy.getSerializer()); - assertEquals(serializer, copy.getSerializer()); - } - - @Test - public void testValueStateDescriptorLazySerializer() throws Exception { - - // some default value that goes to the generic serializer - Path defaultValue = new Path(new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).toURI()); - - // some different registered value - ExecutionConfig cfg = new ExecutionConfig(); - cfg.registerKryoType(TaskInfo.class); - - ValueStateDescriptor descr = - new ValueStateDescriptor<>("testName", Path.class, defaultValue); - - try { - descr.getSerializer(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) {} - - descr.initializeSerializerUnlessSet(cfg); - - assertNotNull(descr.getSerializer()); - assertTrue(descr.getSerializer() instanceof KryoSerializer); - - assertTrue(((KryoSerializer) descr.getSerializer()).getKryo().getRegistration(TaskInfo.class).getId() > 0); - } - - @Test - public void testValueStateDescriptorAutoSerializer() throws Exception { - - String defaultValue = "le-value-default"; - - ValueStateDescriptor descr = - new ValueStateDescriptor<>("testName", String.class, defaultValue); - - ValueStateDescriptor copy = CommonTestUtils.createCopySerializable(descr); - - assertEquals("testName", copy.getName()); - assertEquals(defaultValue, copy.getDefaultValue()); - assertNotNull(copy.getSerializer()); - assertEquals(StringSerializer.INSTANCE, copy.getSerializer()); - } - @Test public void testVeryLargeDefaultValue() throws Exception { // ensure that we correctly read very large data when deserializing the default value From effe7d7becf054c352b5a4f5e9330efc1af2c0da Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Mar 2018 16:46:13 +0100 Subject: [PATCH 185/268] [hotfix] [core] Consolidate serializer duplication tests in StateDescriptorTest where possible --- .../state/AggregatingStateDescriptorTest.java | 62 ------------------- .../common/state/ListStateDescriptorTest.java | 1 - .../state/ReducingStateDescriptorTest.java | 27 -------- .../api/common/state/StateDescriptorTest.java | 30 +++++++++ .../state/ValueStateDescriptorTest.java | 23 ------- 5 files changed, 30 insertions(+), 113 deletions(-) delete mode 100644 flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java deleted file mode 100644 index f62acc8996fd5..0000000000000 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/AggregatingStateDescriptorTest.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.api.common.state; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.functions.AggregateFunction; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import static org.junit.Assert.assertNotSame; -import static org.mockito.Mockito.mock; - -/** - * Tests for the {@link AggregatingStateDescriptor}. - */ -public class AggregatingStateDescriptorTest extends TestLogger { - - /** - * FLINK-6775. - * - *

    Tests that the returned serializer is duplicated. This allows to - * share the state descriptor. - */ - @Test - public void testSerializerDuplication() { - // we need a serializer that actually duplicates for testing (a stateful one) - // we use Kryo here, because it meets these conditions - TypeSerializer serializer = new KryoSerializer<>(Long.class, new ExecutionConfig()); - - AggregateFunction aggregatingFunction = mock(AggregateFunction.class); - - AggregatingStateDescriptor descr = new AggregatingStateDescriptor<>( - "foobar", - aggregatingFunction, - serializer); - - TypeSerializer serializerA = descr.getSerializer(); - TypeSerializer serializerB = descr.getSerializer(); - - // check that the retrieved serializers are not the same - assertNotSame(serializerA, serializerB); - } -} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java index e7e33e79ca578..b934ee09d8ed2 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java @@ -68,7 +68,6 @@ public void testListStateDescriptor() throws Exception { *

    Tests that the returned serializer is duplicated. This allows to * share the state descriptor. */ - @SuppressWarnings("unchecked") @Test public void testSerializerDuplication() { // we need a serializer that actually duplicates for testing (a stateful one) diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java index 81b7c38cf7965..5d9eba5229177 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java @@ -29,7 +29,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; /** * Tests for the {@link ReducingStateDescriptor}. @@ -57,30 +56,4 @@ public void testReducingStateDescriptor() throws Exception { assertNotNull(copy.getSerializer()); assertEquals(serializer, copy.getSerializer()); } - - /** - * FLINK-6775. - * - *

    Tests that the returned serializer is duplicated. This allows to - * share the state descriptor. - */ - @SuppressWarnings("unchecked") - @Test - public void testSerializerDuplication() { - // we need a serializer that actually duplicates for testing (a stateful one) - // we use Kryo here, because it meets these conditions - TypeSerializer statefulSerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); - - ReducingStateDescriptor descr = new ReducingStateDescriptor<>( - "foobar", - (a, b) -> a, - statefulSerializer); - - TypeSerializer serializerA = descr.getSerializer(); - TypeSerializer serializerB = descr.getSerializer(); - - // check that the retrieved serializers are not the same - assertNotSame(serializerA, serializerB); - } - } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java index 59293f410605f..cf5327e49dda4 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java @@ -33,6 +33,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -41,6 +42,10 @@ */ public class StateDescriptorTest { + // ------------------------------------------------------------------------ + // Tests for serializer initialization + // ------------------------------------------------------------------------ + @Test public void testInitializeWithSerializer() throws Exception { final TypeSerializer serializer = StringSerializer.INSTANCE; @@ -130,6 +135,31 @@ public void testInitializeSerializerAfterSerializationWithCustomConfig() throws .getRegistration(File.class).getId() > 0); } + // ------------------------------------------------------------------------ + // Tests for serializer initialization + // ------------------------------------------------------------------------ + + /** + * FLINK-6775, tests that the returned serializer is duplicated. + * This allows to share the state descriptor across threads. + */ + @Test + public void testSerializerDuplication() throws Exception { + // we need a serializer that actually duplicates for testing (a stateful one) + // we use Kryo here, because it meets these conditions + TypeSerializer statefulSerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); + + TestStateDescriptor descr = new TestStateDescriptor<>("foobar", statefulSerializer); + + TypeSerializer serializerA = descr.getSerializer(); + TypeSerializer serializerB = descr.getSerializer(); + + // check that the retrieved serializers are not the same + assertNotSame(serializerA, serializerB); + } + + // ------------------------------------------------------------------------ + // Mock implementations and test types // ------------------------------------------------------------------------ private static class TestStateDescriptor extends StateDescriptor { diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java index 7ee58fedb9864..67114e50af81b 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java @@ -29,7 +29,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNotSame; /** * Tests for the {@link ValueStateDescriptor}. @@ -64,26 +63,4 @@ public void testVeryLargeDefaultValue() throws Exception { assertNotNull(copy.getSerializer()); assertEquals(serializer, copy.getSerializer()); } - - /** - * FLINK-6775. - * - *

    Tests that the returned serializer is duplicated. This allows to - * share the state descriptor. - */ - @SuppressWarnings("unchecked") - @Test - public void testSerializerDuplication() { - // we need a serializer that actually duplicates for testing (a stateful one) - // we use Kryo here, because it meets these conditions - TypeSerializer statefulSerializer = new KryoSerializer<>(String.class, new ExecutionConfig()); - - ValueStateDescriptor descr = new ValueStateDescriptor<>("foobar", statefulSerializer); - - TypeSerializer serializerA = descr.getSerializer(); - TypeSerializer serializerB = descr.getSerializer(); - - // check that the retrieved serializers are not the same - assertNotSame(serializerA, serializerB); - } } From 69e5d146219bbded4bb6cc472ff015996c1aceb7 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 20 Mar 2018 17:16:06 +0100 Subject: [PATCH 186/268] [FLINK-9035] [core] Fix state descriptor equals() and hashCode() handling --- .../state/AggregatingStateDescriptor.java | 31 --------- .../common/state/FoldingStateDescriptor.java | 31 --------- .../api/common/state/ListStateDescriptor.java | 30 -------- .../api/common/state/MapStateDescriptor.java | 29 -------- .../common/state/ReducingStateDescriptor.java | 30 -------- .../api/common/state/StateDescriptor.java | 17 ++++- .../common/state/ValueStateDescriptor.java | 31 --------- .../common/state/ListStateDescriptorTest.java | 28 ++++++++ .../common/state/MapStateDescriptorTest.java | 29 ++++++++ .../state/ReducingStateDescriptorTest.java | 29 ++++++++ .../api/common/state/StateDescriptorTest.java | 69 +++++++++++++++++-- .../state/ValueStateDescriptorTest.java | 28 ++++++++ 12 files changed, 193 insertions(+), 189 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java index 6f6d2f9790e08..8c7fed621da71 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/AggregatingStateDescriptor.java @@ -111,35 +111,4 @@ public AggregateFunction getAggregateFunction() { public Type getType() { return Type.AGGREGATING; } - - // ------------------------------------------------------------------------ - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - else if (o != null && getClass() == o.getClass()) { - AggregatingStateDescriptor that = (AggregatingStateDescriptor) o; - return serializer.equals(that.serializer) && name.equals(that.name); - } - else { - return false; - } - } - - @Override - public int hashCode() { - int result = serializer.hashCode(); - result = 31 * result + name.hashCode(); - return result; - } - - @Override - public String toString() { - return "AggregatingStateDescriptor{" + - "serializer=" + serializer + - ", aggFunction=" + aggFunction + - '}'; - } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java index 261d1fe47219d..c14e4bfc18361 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/FoldingStateDescriptor.java @@ -111,37 +111,6 @@ public FoldFunction getFoldFunction() { return foldFunction; } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - FoldingStateDescriptor that = (FoldingStateDescriptor) o; - - return serializer.equals(that.serializer) && name.equals(that.name); - - } - - @Override - public int hashCode() { - int result = serializer.hashCode(); - result = 31 * result + name.hashCode(); - return result; - } - - @Override - public String toString() { - return "FoldingStateDescriptor{" + - "serializer=" + serializer + - ", initialValue=" + defaultValue + - ", foldFunction=" + foldFunction + - '}'; - } - @Override public Type getType() { return Type.FOLDING; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ListStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ListStateDescriptor.java index 38e56803330e5..aa5e64b019deb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ListStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ListStateDescriptor.java @@ -102,34 +102,4 @@ public TypeSerializer getElementSerializer() { public Type getType() { return Type.LIST; } - - // ------------------------------------------------------------------------ - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - final ListStateDescriptor that = (ListStateDescriptor) o; - return serializer.equals(that.serializer) && name.equals(that.name); - - } - - @Override - public int hashCode() { - int result = serializer.hashCode(); - result = 31 * result + name.hashCode(); - return result; - } - - @Override - public String toString() { - return "ListStateDescriptor{" + - "serializer=" + serializer + - '}'; - } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java index 087cb5410bc59..42b016adc9b14 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/MapStateDescriptor.java @@ -117,33 +117,4 @@ public TypeSerializer getValueSerializer() { return ((MapSerializer) rawSerializer).getValueSerializer(); } - - @Override - public int hashCode() { - int result = serializer.hashCode(); - result = 31 * result + name.hashCode(); - return result; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - MapStateDescriptor that = (MapStateDescriptor) o; - return serializer.equals(that.serializer) && name.equals(that.name); - } - - @Override - public String toString() { - return "MapStateDescriptor{" + - "name=" + name + - ", serializer=" + serializer + - '}'; - } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java index ef483e2af0422..0df1c2c862b06 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ReducingStateDescriptor.java @@ -97,36 +97,6 @@ public ReduceFunction getReduceFunction() { return reduceFunction; } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - ReducingStateDescriptor that = (ReducingStateDescriptor) o; - - return serializer.equals(that.serializer) && name.equals(that.name); - - } - - @Override - public int hashCode() { - int result = serializer.hashCode(); - result = 31 * result + name.hashCode(); - return result; - } - - @Override - public String toString() { - return "ReducingStateDescriptor{" + - "serializer=" + serializer + - ", reduceFunction=" + reduceFunction + - '}'; - } - @Override public Type getType() { return Type.REDUCING; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java index 574c83603ec16..9b6b51dc8ced9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java @@ -273,10 +273,23 @@ public void initializeSerializerUnlessSet(ExecutionConfig executionConfig) { // ------------------------------------------------------------------------ @Override - public abstract int hashCode(); + public final int hashCode() { + return name.hashCode() + 31 * getClass().hashCode(); + } @Override - public abstract boolean equals(Object o); + public final boolean equals(Object o) { + if (o == this) { + return true; + } + else if (o != null && o.getClass() == this.getClass()) { + final StateDescriptor that = (StateDescriptor) o; + return this.name.equals(that.name); + } + else { + return false; + } + } @Override public String toString() { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/ValueStateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ValueStateDescriptor.java index ef18d741209c3..4d69d811e4a59 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/ValueStateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ValueStateDescriptor.java @@ -129,37 +129,6 @@ public ValueState bind(StateBinder stateBinder) throws Exception { return stateBinder.createValueState(this); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - ValueStateDescriptor that = (ValueStateDescriptor) o; - - return serializer.equals(that.serializer) && name.equals(that.name); - - } - - @Override - public int hashCode() { - int result = serializer.hashCode(); - result = 31 * result + name.hashCode(); - return result; - } - - @Override - public String toString() { - return "ValueStateDescriptor{" + - "name=" + name + - ", defaultValue=" + defaultValue + - ", serializer=" + serializer + - '}'; - } - @Override public Type getType() { return Type.VALUE; diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java index b934ee09d8ed2..cb6f6083d7025 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ListStateDescriptorTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.testutils.CommonTestUtils; @@ -62,6 +63,33 @@ public void testListStateDescriptor() throws Exception { assertEquals(serializer, copy.getElementSerializer()); } + @Test + public void testHashCodeEquals() throws Exception { + final String name = "testName"; + + ListStateDescriptor original = new ListStateDescriptor<>(name, String.class); + ListStateDescriptor same = new ListStateDescriptor<>(name, String.class); + ListStateDescriptor sameBySerializer = new ListStateDescriptor<>(name, StringSerializer.INSTANCE); + + // test that hashCode() works on state descriptors with initialized and uninitialized serializers + assertEquals(original.hashCode(), same.hashCode()); + assertEquals(original.hashCode(), sameBySerializer.hashCode()); + + assertEquals(original, same); + assertEquals(original, sameBySerializer); + + // equality with a clone + ListStateDescriptor clone = CommonTestUtils.createCopySerializable(original); + assertEquals(original, clone); + + // equality with an initialized + clone.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, clone); + + original.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, same); + } + /** * FLINK-6775. * diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java index 4e64c0f436a01..069d6c2ae07d7 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/MapStateDescriptorTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.testutils.CommonTestUtils; @@ -67,6 +68,34 @@ public void testMapStateDescriptor() throws Exception { assertEquals(valueSerializer, copy.getValueSerializer()); } + @Test + public void testHashCodeEquals() throws Exception { + final String name = "testName"; + + MapStateDescriptor original = new MapStateDescriptor<>(name, String.class, String.class); + MapStateDescriptor same = new MapStateDescriptor<>(name, String.class, String.class); + MapStateDescriptor sameBySerializer = + new MapStateDescriptor<>(name, StringSerializer.INSTANCE, StringSerializer.INSTANCE); + + // test that hashCode() works on state descriptors with initialized and uninitialized serializers + assertEquals(original.hashCode(), same.hashCode()); + assertEquals(original.hashCode(), sameBySerializer.hashCode()); + + assertEquals(original, same); + assertEquals(original, sameBySerializer); + + // equality with a clone + MapStateDescriptor clone = CommonTestUtils.createCopySerializable(original); + assertEquals(original, clone); + + // equality with an initialized + clone.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, clone); + + original.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, same); + } + /** * FLINK-6775. * diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java index 5d9eba5229177..89aa1e688242b 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ReducingStateDescriptorTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.util.TestLogger; @@ -56,4 +57,32 @@ public void testReducingStateDescriptor() throws Exception { assertNotNull(copy.getSerializer()); assertEquals(serializer, copy.getSerializer()); } + + @Test + public void testHashCodeEquals() throws Exception { + final String name = "testName"; + final ReduceFunction reducer = (a, b) -> a; + + ReducingStateDescriptor original = new ReducingStateDescriptor<>(name, reducer, String.class); + ReducingStateDescriptor same = new ReducingStateDescriptor<>(name, reducer, String.class); + ReducingStateDescriptor sameBySerializer = new ReducingStateDescriptor<>(name, reducer, StringSerializer.INSTANCE); + + // test that hashCode() works on state descriptors with initialized and uninitialized serializers + assertEquals(original.hashCode(), same.hashCode()); + assertEquals(original.hashCode(), sameBySerializer.hashCode()); + + assertEquals(original, same); + assertEquals(original, sameBySerializer); + + // equality with a clone + ReducingStateDescriptor clone = CommonTestUtils.createCopySerializable(original); + assertEquals(original, clone); + + // equality with an initialized + clone.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, clone); + + original.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, same); + } } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java index cf5327e49dda4..3958baa120a12 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/StateDescriptorTest.java @@ -32,6 +32,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; @@ -158,6 +159,47 @@ public void testSerializerDuplication() throws Exception { assertNotSame(serializerA, serializerB); } + // ------------------------------------------------------------------------ + // Test hashCode() and equals() + // ------------------------------------------------------------------------ + + @Test + public void testHashCodeAndEquals() throws Exception { + final String name = "testName"; + + TestStateDescriptor original = new TestStateDescriptor<>(name, String.class); + TestStateDescriptor same = new TestStateDescriptor<>(name, String.class); + TestStateDescriptor sameBySerializer = new TestStateDescriptor<>(name, StringSerializer.INSTANCE); + + // test that hashCode() works on state descriptors with initialized and uninitialized serializers + assertEquals(original.hashCode(), same.hashCode()); + assertEquals(original.hashCode(), sameBySerializer.hashCode()); + + assertEquals(original, same); + assertEquals(original, sameBySerializer); + + // equality with a clone + TestStateDescriptor clone = CommonTestUtils.createCopySerializable(original); + assertEquals(original, clone); + + // equality with an initialized + clone.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, clone); + + original.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, same); + } + + @Test + public void testEqualsSameNameAndTypeDifferentClass() throws Exception { + final String name = "test name"; + + final TestStateDescriptor descr1 = new TestStateDescriptor<>(name, String.class); + final OtherTestStateDescriptor descr2 = new OtherTestStateDescriptor<>(name, String.class); + + assertNotEquals(descr1, descr2); + } + // ------------------------------------------------------------------------ // Mock implementations and test types // ------------------------------------------------------------------------ @@ -185,17 +227,34 @@ public State bind(StateBinder stateBinder) throws Exception { @Override public Type getType() { - throw new UnsupportedOperationException(); + return Type.VALUE; + } + } + + private static class OtherTestStateDescriptor extends StateDescriptor { + + private static final long serialVersionUID = 1L; + + OtherTestStateDescriptor(String name, TypeSerializer serializer) { + super(name, serializer, null); + } + + OtherTestStateDescriptor(String name, TypeInformation typeInfo) { + super(name, typeInfo, null); + } + + OtherTestStateDescriptor(String name, Class type) { + super(name, type, null); } @Override - public int hashCode() { - return 584523; + public State bind(StateBinder stateBinder) throws Exception { + throw new UnsupportedOperationException(); } @Override - public boolean equals(Object o) { - return o != null && o.getClass() == TestStateDescriptor.class; + public Type getType() { + return Type.VALUE; } } } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java index 67114e50af81b..3870da05da242 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/state/ValueStateDescriptorTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.core.testutils.CommonTestUtils; @@ -35,6 +36,33 @@ */ public class ValueStateDescriptorTest extends TestLogger { + @Test + public void testHashCodeEquals() throws Exception { + final String name = "testName"; + + ValueStateDescriptor original = new ValueStateDescriptor<>(name, String.class); + ValueStateDescriptor same = new ValueStateDescriptor<>(name, String.class); + ValueStateDescriptor sameBySerializer = new ValueStateDescriptor<>(name, StringSerializer.INSTANCE); + + // test that hashCode() works on state descriptors with initialized and uninitialized serializers + assertEquals(original.hashCode(), same.hashCode()); + assertEquals(original.hashCode(), sameBySerializer.hashCode()); + + assertEquals(original, same); + assertEquals(original, sameBySerializer); + + // equality with a clone + ValueStateDescriptor clone = CommonTestUtils.createCopySerializable(original); + assertEquals(original, clone); + + // equality with an initialized + clone.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, clone); + + original.initializeSerializerUnlessSet(new ExecutionConfig()); + assertEquals(original, same); + } + @Test public void testVeryLargeDefaultValue() throws Exception { // ensure that we correctly read very large data when deserializing the default value From 4b66514f2c31d5ea29493baf9d022a0115faf82d Mon Sep 17 00:00:00 2001 From: Shuyi Chen Date: Tue, 6 Feb 2018 00:50:21 -0800 Subject: [PATCH 187/268] [FLINK-8562] [tests] Fix YARNSessionFIFOSecuredITCase Before the YARNSessionFIFOSecuredITCase also passed without Kerberos being active. This closes #5416. --- .../flink/yarn/YARNSessionFIFOITCase.java | 13 ++- .../yarn/YARNSessionFIFOSecuredITCase.java | 13 +++ .../flink/yarn/YarnConfigurationITCase.java | 2 +- .../org/apache/flink/yarn/YarnTestBase.java | 79 ++++++++++++++----- 4 files changed, 86 insertions(+), 21 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index b3dcaca145965..464e73c873e4e 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -49,6 +49,7 @@ import java.util.Arrays; import java.util.EnumSet; import java.util.List; +import java.util.concurrent.TimeUnit; import static org.apache.flink.yarn.UtilsTest.addTestAppender; import static org.apache.flink.yarn.UtilsTest.checkForLogString; @@ -106,8 +107,16 @@ public void testDetachedMode() throws InterruptedException, IOException { } } - //additional sleep for the JM/TM to start and establish connection - sleep(2000); + // additional sleep for the JM/TM to start and establish connection + long startTime = System.nanoTime(); + while (System.nanoTime() - startTime < TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS) && + !(verifyStringsInNamedLogFiles( + new String[]{"YARN Application Master started"}, "jobmanager.log") && + verifyStringsInNamedLogFiles( + new String[]{"Starting TaskManager actor"}, "taskmanager.log"))) { + LOG.info("Still waiting for JM/TM to initialize..."); + sleep(500); + } LOG.info("Two containers are running. Killing the application"); // kill application "externally". diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java index 3954f8ab73e52..18e1c3abff0db 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java @@ -30,10 +30,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.junit.AfterClass; +import org.junit.Assert; import org.junit.BeforeClass; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.Collections; import java.util.concurrent.Callable; @@ -97,6 +99,17 @@ public static void teardownSecureCluster() throws Exception { SecureTestEnvironment.cleanup(); } + @Override + public void testDetachedMode() throws InterruptedException, IOException { + super.testDetachedMode(); + if (!verifyStringsInNamedLogFiles( + new String[]{"Login successful for user", "using keytab file"}, "jobmanager.log") || + !verifyStringsInNamedLogFiles( + new String[]{"Login successful for user", "using keytab file"}, "taskmanager.log")) { + Assert.fail("Can not find expected strings in log files."); + } + } + /* For secure cluster testing, it is enough to run only one test and override below test methods * to keep the overall build time minimal */ diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java index 2a1b099399ac0..635fdf3d0959a 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java @@ -78,7 +78,7 @@ public class YarnConfigurationITCase extends YarnTestBase { @Test(timeout = 60000) public void testFlinkContainerMemory() throws Exception { final YarnClient yarnClient = getYarnClient(); - final Configuration configuration = new Configuration(flinkConfiguration); + final Configuration configuration = new Configuration(flinkConfiguration.clone()); final int masterMemory = 64; final int taskManagerMemory = 128; diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 3ec805e5058c5..803f89cd931b5 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -21,7 +21,9 @@ import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -57,7 +59,6 @@ import javax.annotation.Nullable; -import java.io.BufferedWriter; import java.io.ByteArrayOutputStream; import java.io.File; import java.io.FileNotFoundException; @@ -68,17 +69,20 @@ import java.io.PipedInputStream; import java.io.PipedOutputStream; import java.io.PrintStream; -import java.io.PrintWriter; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Scanner; +import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.regex.Pattern; +import static org.apache.flink.configuration.CoreOptions.OLD_MODE; + /** * This base class allows to use the MiniYARNCluster. * The cluster is re-used for all tests. @@ -145,7 +149,7 @@ public abstract class YarnTestBase extends TestLogger { private YarnClient yarnClient = null; - protected org.apache.flink.configuration.Configuration flinkConfiguration; + protected static org.apache.flink.configuration.Configuration flinkConfiguration; protected boolean flip6; @@ -213,8 +217,6 @@ public void checkClusterEmpty() throws IOException, YarnException { } } - flinkConfiguration = new org.apache.flink.configuration.Configuration(); - flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(flinkConfiguration.getString(CoreOptions.MODE)); } @@ -397,6 +399,51 @@ public boolean accept(File dir, String name) { } } + public static boolean verifyStringsInNamedLogFiles( + final String[] mustHave, final String fileName) { + List mustHaveList = Arrays.asList(mustHave); + File cwd = new File("target/" + YARN_CONFIGURATION.get(TEST_CLUSTER_NAME_KEY)); + if (!cwd.exists() || !cwd.isDirectory()) { + return false; + } + + File foundFile = findFile(cwd.getAbsolutePath(), new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + if (fileName != null && !name.equals(fileName)) { + return false; + } + File f = new File(dir.getAbsolutePath() + "/" + name); + LOG.info("Searching in {}", f.getAbsolutePath()); + try { + Set foundSet = new HashSet<>(mustHave.length); + Scanner scanner = new Scanner(f); + while (scanner.hasNextLine()) { + final String lineFromFile = scanner.nextLine(); + for (String str : mustHave) { + if (lineFromFile.contains(str)) { + foundSet.add(str); + } + } + if (foundSet.containsAll(mustHaveList)) { + return true; + } + } + } catch (FileNotFoundException e) { + LOG.warn("Unable to locate file: " + e.getMessage() + " file: " + f.getAbsolutePath()); + } + return false; + } + }); + + if (foundFile != null) { + LOG.info("Found string {} in {}.", Arrays.toString(mustHave), foundFile.getAbsolutePath()); + return true; + } else { + return false; + } + } + public static void sleep(int time) { try { Thread.sleep(time); @@ -465,27 +512,23 @@ private static void start(YarnConfiguration conf, String principal, String keyta File flinkConfDirPath = findFile(flinkDistRootDir, new ContainsName(new String[]{"flink-conf.yaml"})); Assert.assertNotNull(flinkConfDirPath); + flinkConfiguration = + GlobalConfiguration.loadConfiguration(); if (!StringUtils.isBlank(principal) && !StringUtils.isBlank(keytab)) { + //copy conf dir to test temporary workspace location tempConfPathForSecureRun = tmp.newFolder("conf"); String confDirPath = flinkConfDirPath.getParentFile().getAbsolutePath(); FileUtils.copyDirectory(new File(confDirPath), tempConfPathForSecureRun); - try (FileWriter fw = new FileWriter(new File(tempConfPathForSecureRun, "flink-conf.yaml"), true); - BufferedWriter bw = new BufferedWriter(fw); - PrintWriter out = new PrintWriter(bw)) { - - LOG.info("writing keytab: " + keytab + " and principal: " + principal + " to config file"); - out.println(""); - out.println("#Security Configurations Auto Populated "); - out.println(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key() + ": " + keytab); - out.println(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key() + ": " + principal); - out.println(""); - } catch (IOException e) { - throw new RuntimeException("Exception occured while trying to append the security configurations.", e); - } + flinkConfiguration.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key(), keytab); + flinkConfiguration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key(), principal); + flinkConfiguration.setString(CoreOptions.MODE.key(), OLD_MODE); + + BootstrapTools.writeConfiguration(flinkConfiguration, + new File(tempConfPathForSecureRun, "flink-conf.yaml")); String configDir = tempConfPathForSecureRun.getAbsolutePath(); From 9c105f2c982e511bc1274a86d629e7fa26cf7ac8 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 22 Mar 2018 13:16:10 +0100 Subject: [PATCH 188/268] [FLINK-8562] [tests] Introduce private global configuration to YarnTestBase --- .../yarn/YARNSessionFIFOSecuredITCase.java | 18 ++++++++++----- .../flink/yarn/YarnConfigurationITCase.java | 2 +- .../org/apache/flink/yarn/YarnTestBase.java | 22 +++++++++++-------- 3 files changed, 26 insertions(+), 16 deletions(-) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java index 18e1c3abff0db..46a37a0f8fdf8 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOSecuredITCase.java @@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; +import org.hamcrest.Matchers; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -102,12 +103,17 @@ public static void teardownSecureCluster() throws Exception { @Override public void testDetachedMode() throws InterruptedException, IOException { super.testDetachedMode(); - if (!verifyStringsInNamedLogFiles( - new String[]{"Login successful for user", "using keytab file"}, "jobmanager.log") || - !verifyStringsInNamedLogFiles( - new String[]{"Login successful for user", "using keytab file"}, "taskmanager.log")) { - Assert.fail("Can not find expected strings in log files."); - } + final String[] mustHave = {"Login successful for user", "using keytab file"}; + final boolean jobManagerRunsWithKerberos = verifyStringsInNamedLogFiles( + mustHave, + "jobmanager.log"); + final boolean taskManagerRunsWithKerberos = verifyStringsInNamedLogFiles( + mustHave, "taskmanager.log"); + + Assert.assertThat( + "The JobManager and the TaskManager should both run with Kerberos.", + jobManagerRunsWithKerberos && taskManagerRunsWithKerberos, + Matchers.is(true)); } /* For secure cluster testing, it is enough to run only one test and override below test methods diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java index 635fdf3d0959a..2a1b099399ac0 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java @@ -78,7 +78,7 @@ public class YarnConfigurationITCase extends YarnTestBase { @Test(timeout = 60000) public void testFlinkContainerMemory() throws Exception { final YarnClient yarnClient = getYarnClient(); - final Configuration configuration = new Configuration(flinkConfiguration.clone()); + final Configuration configuration = new Configuration(flinkConfiguration); final int masterMemory = 64; final int taskManagerMemory = 128; diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 803f89cd931b5..73abc874b915f 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -149,7 +149,9 @@ public abstract class YarnTestBase extends TestLogger { private YarnClient yarnClient = null; - protected static org.apache.flink.configuration.Configuration flinkConfiguration; + private static org.apache.flink.configuration.Configuration globalConfiguration; + + protected org.apache.flink.configuration.Configuration flinkConfiguration; protected boolean flip6; @@ -217,6 +219,7 @@ public void checkClusterEmpty() throws IOException, YarnException { } } + flinkConfiguration = new org.apache.flink.configuration.Configuration(globalConfiguration); flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(flinkConfiguration.getString(CoreOptions.MODE)); } @@ -512,23 +515,24 @@ private static void start(YarnConfiguration conf, String principal, String keyta File flinkConfDirPath = findFile(flinkDistRootDir, new ContainsName(new String[]{"flink-conf.yaml"})); Assert.assertNotNull(flinkConfDirPath); - flinkConfiguration = - GlobalConfiguration.loadConfiguration(); + + final String confDirPath = flinkConfDirPath.getParentFile().getAbsolutePath(); + globalConfiguration = GlobalConfiguration.loadConfiguration(confDirPath); if (!StringUtils.isBlank(principal) && !StringUtils.isBlank(keytab)) { //copy conf dir to test temporary workspace location tempConfPathForSecureRun = tmp.newFolder("conf"); - String confDirPath = flinkConfDirPath.getParentFile().getAbsolutePath(); FileUtils.copyDirectory(new File(confDirPath), tempConfPathForSecureRun); - flinkConfiguration.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key(), keytab); - flinkConfiguration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key(), principal); - flinkConfiguration.setString(CoreOptions.MODE.key(), OLD_MODE); + globalConfiguration.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key(), keytab); + globalConfiguration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key(), principal); + globalConfiguration.setString(CoreOptions.MODE.key(), OLD_MODE); - BootstrapTools.writeConfiguration(flinkConfiguration, - new File(tempConfPathForSecureRun, "flink-conf.yaml")); + BootstrapTools.writeConfiguration( + globalConfiguration, + new File(tempConfPathForSecureRun, "flink-conf.yaml")); String configDir = tempConfPathForSecureRun.getAbsolutePath(); From 6e23a88d43807fcab0c2256abef4bbc9af9324a6 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 22 Mar 2018 17:15:36 +0100 Subject: [PATCH 189/268] [hotfix] Remove unnecessary transient modifiers in CheckpointStatsTracker --- .../flink/runtime/checkpoint/CheckpointStatsTracker.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java index e6386ad74d624..9be1f69581403 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java @@ -59,7 +59,7 @@ public class CheckpointStatsTracker { * from a single Thread at a time and there can be multiple concurrent read * accesses to the latest stats snapshot. * - * Currently, writes are executed by whatever Thread executes the coordinator + *

    Currently, writes are executed by whatever Thread executes the coordinator * actions (which already happens in locked scope). Reads can come from * multiple concurrent Netty event loop Threads of the web runtime monitor. */ @@ -81,7 +81,7 @@ public class CheckpointStatsTracker { private final CheckpointStatsHistory history; /** The job vertices taking part in the checkpoints. */ - private final transient List jobVertices; + private final List jobVertices; /** The latest restored checkpoint. */ @Nullable @@ -99,7 +99,7 @@ public class CheckpointStatsTracker { /** The latest completed checkpoint. Used by the latest completed checkpoint metrics. */ @Nullable - private volatile transient CompletedCheckpointStats latestCompletedCheckpoint; + private volatile CompletedCheckpointStats latestCompletedCheckpoint; /** * Creates a new checkpoint stats tracker. From 2344ce2b51b959fd53caf9a372f23f01518a873d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 22 Mar 2018 18:38:56 +0100 Subject: [PATCH 190/268] [hotfix] Add generics to FutureUtils.toJava calls in ClusterClient --- .../org/apache/flink/client/program/ClusterClient.java | 10 +++++----- .../apache/flink/runtime/concurrent/FutureUtils.java | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index b0c50e59d7705..166d9770b48bb 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -602,7 +602,7 @@ public CompletableFuture getJobStatus(JobID jobId) { Future response = jobManager.ask(JobManagerMessages.getRequestJobStatus(jobId), timeout); - CompletableFuture javaFuture = FutureUtils.toJava(response); + CompletableFuture javaFuture = FutureUtils.toJava(response); return javaFuture.thenApply((responseMessage) -> { if (responseMessage instanceof JobManagerMessages.CurrentJobStatus) { @@ -707,9 +707,9 @@ public void stop(final JobID jobId) throws Exception { public CompletableFuture triggerSavepoint(JobID jobId, @Nullable String savepointDirectory) throws FlinkException { final ActorGateway jobManager = getJobManagerGateway(); - Future response = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobId, Option.apply(savepointDirectory)), + Future response = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobId, Option.apply(savepointDirectory)), new FiniteDuration(1, TimeUnit.HOURS)); - CompletableFuture responseFuture = FutureUtils.toJava(response); + CompletableFuture responseFuture = FutureUtils.toJava(response); return responseFuture.thenApply((responseMessage) -> { if (responseMessage instanceof JobManagerMessages.TriggerSavepointSuccess) { @@ -729,7 +729,7 @@ public CompletableFuture disposeSavepoint(String savepointPath, Tim final ActorGateway jobManager = getJobManagerGateway(); Object msg = new JobManagerMessages.DisposeSavepoint(savepointPath); - CompletableFuture responseFuture = FutureUtils.toJava( + CompletableFuture responseFuture = FutureUtils.toJava( jobManager.ask( msg, FutureUtils.toFiniteDuration(timeout))); @@ -768,7 +768,7 @@ public CompletableFuture> listJobs() throws Excepti final ActorGateway jobManager = getJobManagerGateway(); Future response = jobManager.ask(new RequestJobDetails(true, false), timeout); - CompletableFuture responseFuture = FutureUtils.toJava(response); + CompletableFuture responseFuture = FutureUtils.toJava(response); return responseFuture.thenApply((responseMessage) -> { if (responseMessage instanceof MultipleJobsDetails) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index a2d0710e87977..e0164a92f6a99 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -750,7 +750,7 @@ public static CompletableFuture toJava(Future scalaFuture) { scalaFuture.onComplete(new OnComplete() { @Override - public void onComplete(Throwable failure, T success) throws Throwable { + public void onComplete(Throwable failure, T success) { if (failure != null) { result.completeExceptionally(failure); } else { From 69114a4f9b6dbbda2929b88903dda8a91a0f49fe Mon Sep 17 00:00:00 2001 From: yanghua Date: Thu, 22 Mar 2018 09:58:08 +0800 Subject: [PATCH 191/268] [FLINK-8931] TASK_KILLING is not covered by match in TaskMonitor#whenUnhandled This closes #5744. --- .../scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala index 7840fd479bd24..76a2a90e1fc07 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala @@ -163,6 +163,7 @@ class TaskMonitor( LOG.warn(s"Mesos task ${goal.taskID.getValue} failed unexpectedly.") context.parent ! TaskTerminated(goal.taskID, msg.status()) stop() + case TASK_KILLING => stay() } case Event(msg: StatusUpdate, StateData(goal: Released)) => @@ -175,6 +176,7 @@ class TaskMonitor( LOG.info(s"Mesos task ${goal.taskID.getValue} exited as planned.") context.parent ! TaskTerminated(goal.taskID, msg.status()) stop() + case TASK_KILLING => stay() } } From 3bf5582529cfe9a65c899749b08052a978875689 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Mar 2018 15:09:46 +0100 Subject: [PATCH 192/268] [hotfix] Improve Flip-6 component logging --- .../flink/runtime/jobmaster/JobMaster.java | 4 ++-- .../runtime/jobmaster/slotpool/SlotPool.java | 5 +++-- .../resourcemanager/ResourceManager.java | 17 ++++++++--------- .../slotmanager/ResourceActions.java | 3 ++- .../slotmanager/SlotManager.java | 8 ++++---- .../runtime/taskexecutor/TaskExecutor.java | 2 ++ .../slotmanager/SlotManagerTest.java | 8 ++++---- 7 files changed, 25 insertions(+), 22 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java index 6878032f0e17f..bc18fab7e39ad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java @@ -689,6 +689,8 @@ public CompletableFuture scheduleOrUpdateConsumers( @Override public CompletableFuture disconnectTaskManager(final ResourceID resourceID, final Exception cause) { + log.debug("Disconnect TaskExecutor {} because: {}", resourceID, cause.getMessage()); + taskManagerHeartbeatManager.unmonitorTarget(resourceID); CompletableFuture releaseFuture = slotPoolGateway.releaseTaskManager(resourceID); @@ -1516,8 +1518,6 @@ private TaskManagerHeartbeatListener(JobMasterGateway jobMasterGateway) { @Override public void notifyHeartbeatTimeout(ResourceID resourceID) { - log.info("Heartbeat of TaskManager with id {} timed out.", resourceID); - jobMasterGateway.disconnectTaskManager( resourceID, new TimeoutException("Heartbeat of TaskManager with id " + resourceID + " timed out.")); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index 42264b5321989..3e8b788239547 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -322,7 +322,6 @@ private CompletableFuture internalAllocateSlot( SlotProfile slotProfile, boolean allowQueuedScheduling, Time allocationTimeout) { - final SlotSharingGroupId slotSharingGroupId = task.getSlotSharingGroupId(); if (slotSharingGroupId != null) { @@ -955,7 +954,7 @@ public CompletableFuture offerSlot( } final AllocatedSlot allocatedSlot = new AllocatedSlot( - slotOffer.getAllocationId(), + allocationID, taskManagerLocation, slotOffer.getSlotIndex(), slotOffer.getResourceProfile(), @@ -971,6 +970,8 @@ public CompletableFuture offerSlot( // we could not complete the pending slot future --> try to fulfill another pending request allocatedSlots.remove(pendingRequest.getSlotRequestId()); tryFulfillSlotRequestOrMakeAvailable(allocatedSlot); + } else { + log.debug("Fulfilled slot request {} with allocated slot {}.", pendingRequest.getSlotRequestId(), allocationID); } } else { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index 0ae4ab6af3a08..cae9c6cdb383c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -682,7 +682,7 @@ private RegistrationResponse registerTaskExecutorInternal( WorkerRegistration oldRegistration = taskExecutors.remove(taskExecutorResourceId); if (oldRegistration != null) { // TODO :: suggest old taskExecutor to stop itself - log.info("Replacing old instance of worker for ResourceID {}", taskExecutorResourceId); + log.info("Replacing old registration of TaskExecutor {}.", taskExecutorResourceId); // remove old task manager registration from slot manager slotManager.unregisterTaskManager(oldRegistration.getInstanceID()); @@ -779,14 +779,14 @@ protected void closeTaskManagerConnection(final ResourceID resourceID, final Exc WorkerRegistration workerRegistration = taskExecutors.remove(resourceID); if (workerRegistration != null) { - log.info("Task manager {} failed because {}.", resourceID, cause.getMessage()); + log.info("Closing TaskExecutor connection {} because: {}", resourceID, cause.getMessage()); // TODO :: suggest failed task executor to stop itself slotManager.unregisterTaskManager(workerRegistration.getInstanceID()); workerRegistration.getTaskExecutorGateway().disconnectResourceManager(cause); } else { - log.debug("Could not find a registered task manager with the process id {}.", resourceID); + log.debug("No open TaskExecutor connection {}. Ignoring close TaskExecutor connection.", resourceID); } } @@ -816,7 +816,7 @@ protected void jobLeaderLostLeadership(JobID jobId, JobMasterId oldJobMasterId) } } - protected void releaseResource(InstanceID instanceId) { + protected void releaseResource(InstanceID instanceId, Exception cause) { WorkerType worker = null; // TODO: Improve performance by having an index on the instanceId @@ -829,10 +829,9 @@ protected void releaseResource(InstanceID instanceId) { if (worker != null) { if (stopWorker(worker)) { - closeTaskManagerConnection(worker.getResourceID(), - new FlinkException("Worker was stopped.")); + closeTaskManagerConnection(worker.getResourceID(), cause); } else { - log.debug("Worker {} was not stopped.", worker.getResourceID()); + log.debug("Worker {} could not be stopped.", worker.getResourceID()); } } else { // unregister in order to clean up potential left over state @@ -990,10 +989,10 @@ protected abstract void shutDownApplication( private class ResourceActionsImpl implements ResourceActions { @Override - public void releaseResource(InstanceID instanceId) { + public void releaseResource(InstanceID instanceId, Exception cause) { validateRunsInMainThread(); - ResourceManager.this.releaseResource(instanceId); + ResourceManager.this.releaseResource(instanceId, cause); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceActions.java index 753e5e2a09218..84e7c4e785d48 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceActions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/ResourceActions.java @@ -33,8 +33,9 @@ public interface ResourceActions { * Releases the resource with the given instance id. * * @param instanceId identifying which resource to release + * @param cause why the resource is released */ - void releaseResource(InstanceID instanceId); + void releaseResource(InstanceID instanceId, Exception cause); /** * Requests to allocate a resource with the given {@link ResourceProfile}. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java index 120e1aa1a5ac3..6cdd997c38d03 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManager.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException; import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -879,8 +880,6 @@ private void checkTaskManagerTimeouts() { // first retrieve the timed out TaskManagers for (TaskManagerRegistration taskManagerRegistration : taskManagerRegistrations.values()) { - LOG.debug("Evaluating TaskManager {} for idleness.", taskManagerRegistration.getInstanceId()); - if (currentTime - taskManagerRegistration.getIdleSince() >= taskManagerTimeout.toMilliseconds()) { // we collect the instance ids first in order to avoid concurrent modifications by the // ResourceActions.releaseResource call @@ -890,7 +889,8 @@ private void checkTaskManagerTimeouts() { // second we trigger the release resource callback which can decide upon the resource release for (InstanceID timedOutTaskManagerId : timedOutTaskManagerIds) { - resourceActions.releaseResource(timedOutTaskManagerId); + LOG.debug("Release TaskExecutor {} because it exceeded the idle timeout.", timedOutTaskManagerId); + resourceActions.releaseResource(timedOutTaskManagerId, new FlinkException("TaskExecutor exceeded the idle timeout.")); } } } @@ -976,7 +976,7 @@ public void unregisterTaskManagersAndReleaseResources() { internalUnregisterTaskManager(taskManagerRegistration); - resourceActions.releaseResource(taskManagerRegistration.getInstanceId()); + resourceActions.releaseResource(taskManagerRegistration.getInstanceId(), new FlinkException("Triggering of SlotManager#unregisterTaskManagersAndReleaseResources.")); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index f25601e534b0c..3523992e2d405 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -1262,6 +1262,8 @@ private void unregisterTaskAndNotifyFinalState( private void freeSlotInternal(AllocationID allocationId, Throwable cause) { checkNotNull(allocationId); + log.debug("Free slot with allocation id {} because: {}", allocationId, cause.getMessage()); + try { final JobID jobId = taskSlotTable.getOwningJob(allocationId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java index 4907756e910d3..90ed1648ad181 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java @@ -685,7 +685,7 @@ public void run() { }); verify(resourceManagerActions, timeout(100L * tmTimeout).times(1)) - .releaseResource(eq(taskManagerConnection.getInstanceID())); + .releaseResource(eq(taskManagerConnection.getInstanceID()), any(Exception.class)); } } @@ -1027,13 +1027,13 @@ public void testTimeoutForUnusedTaskManager() throws Exception { assertTrue(idleFuture2.get()); - verify(resourceManagerActions, timeout(verifyTimeout).times(1)).releaseResource(eq(taskManagerConnection.getInstanceID())); + verify(resourceManagerActions, timeout(verifyTimeout).times(1)).releaseResource(eq(taskManagerConnection.getInstanceID()), any(Exception.class)); } } /** * Tests that a task manager timeout does not remove the slots from the SlotManager. - * A timeout should only trigger the {@link ResourceActions#releaseResource(InstanceID)} + * A timeout should only trigger the {@link ResourceActions#releaseResource(InstanceID, Exception)} * callback. The receiver of the callback can then decide what to do with the TaskManager. * * FLINK-7793 @@ -1064,7 +1064,7 @@ public void testTaskManagerTimeoutDoesNotRemoveSlots() throws Exception { assertEquals(1, slotManager.getNumberRegisteredSlots()); // wait for the timeout call to happen - verify(resourceActions, timeout(taskManagerTimeout.toMilliseconds() * 20L).atLeast(1)).releaseResource(eq(taskExecutorConnection.getInstanceID())); + verify(resourceActions, timeout(taskManagerTimeout.toMilliseconds() * 20L).atLeast(1)).releaseResource(eq(taskExecutorConnection.getInstanceID()), any(Exception.class)); assertEquals(1, slotManager.getNumberRegisteredSlots()); From c28828591073399ef1585f16a3a76fbf327e7be4 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Mar 2018 16:48:52 +0100 Subject: [PATCH 193/268] [FLINK-9047] Fix slot recycling in case of failed release In case that a slot cannot be released it will only recycled/reused if the owning TaskExecutor is still registered at the SlotPool. If this is not the case then we drop the slot from the SlotPool. This closes #5739. --- .../jobmaster/slotpool/SingleLogicalSlot.java | 10 +- .../runtime/jobmaster/slotpool/SlotPool.java | 12 +- .../utils/SimpleAckingTaskManagerGateway.java | 18 +-- .../jobmaster/slotpool/SlotPoolTest.java | 120 +++++++++++++++++- 4 files changed, 139 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SingleLogicalSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SingleLogicalSlot.java index 9bd559bc166ef..0736b5684ab5b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SingleLogicalSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SingleLogicalSlot.java @@ -19,13 +19,13 @@ package org.apache.flink.runtime.jobmaster.slotpool; import org.apache.flink.runtime.clusterframework.types.AllocationID; +import org.apache.flink.runtime.instance.SlotSharingGroupId; +import org.apache.flink.runtime.jobmanager.scheduler.Locality; +import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.SlotContext; import org.apache.flink.runtime.jobmaster.SlotOwner; import org.apache.flink.runtime.jobmaster.SlotRequestId; -import org.apache.flink.runtime.instance.SlotSharingGroupId; -import org.apache.flink.runtime.jobmanager.scheduler.Locality; -import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.Preconditions; @@ -33,7 +33,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; -import java.util.function.Function; /** * Implementation of the {@link LogicalSlot} which is used by the {@link SlotPool}. @@ -127,8 +126,7 @@ public CompletableFuture releaseSlot(@Nullable Throwable cause) { // Wait until the payload has been terminated. Only then, we return the slot to its rightful owner return payload.getTerminalStateFuture() - .handle((Object ignored, Throwable throwable) -> slotOwner.returnAllocatedSlot(this)) - .thenApply(Function.identity()); + .whenComplete((Object ignored, Throwable throwable) -> slotOwner.returnAllocatedSlot(this)); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index 3e8b788239547..6040b41833695 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -1041,6 +1041,7 @@ else if (availableSlots.tryRemove(allocationID)) { */ @Override public CompletableFuture registerTaskManager(final ResourceID resourceID) { + log.debug("Register new TaskExecutor {}.", resourceID); registeredTaskManagers.add(resourceID); return CompletableFuture.completedFuture(Acknowledge.get()); @@ -1119,8 +1120,15 @@ private void checkIdleSlot() { freeSlotFuture.whenCompleteAsync( (Acknowledge ignored, Throwable throwable) -> { if (throwable != null) { - log.info("Releasing idle slot {} failed.", allocationID, throwable); - tryFulfillSlotRequestOrMakeAvailable(expiredSlot); + if (registeredTaskManagers.contains(expiredSlot.getTaskManagerId())) { + log.debug("Releasing slot {} of registered TaskExecutor {} failed. " + + "Trying to fulfill a different slot request.", allocationID, expiredSlot.getTaskManagerId(), + throwable); + tryFulfillSlotRequestOrMakeAvailable(expiredSlot); + } else { + log.debug("Releasing slot {} failed and owning TaskExecutor {} is no " + + "longer registered. Discarding slot.", allocationID, expiredSlot.getTaskManagerId()); + } } }, getMainThreadExecutor()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java index 628f0041371e5..5c62a7370967c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/utils/SimpleAckingTaskManagerGateway.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.blob.TransientBlobKey; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; @@ -38,6 +37,7 @@ import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.function.BiFunction; import java.util.function.Consumer; /** @@ -52,7 +52,7 @@ public class SimpleAckingTaskManagerGateway implements TaskManagerGateway { private Optional> optCancelConsumer; - private volatile Consumer> freeSlotConsumer; + private volatile BiFunction> freeSlotFunction; public SimpleAckingTaskManagerGateway() { optSubmitConsumer = Optional.empty(); @@ -67,8 +67,8 @@ public void setCancelConsumer(Consumer predicate) { optCancelConsumer = Optional.of(predicate); } - public void setFreeSlotConsumer(Consumer> consumer) { - freeSlotConsumer = consumer; + public void setFreeSlotFunction(BiFunction> freeSlotFunction) { + this.freeSlotFunction = freeSlotFunction; } @Override @@ -150,12 +150,12 @@ public CompletableFuture requestTaskManagerStdout(Time timeout @Override public CompletableFuture freeSlot(AllocationID allocationId, Throwable cause, Time timeout) { - final Consumer> currentFreeSlotConsumer = freeSlotConsumer; + final BiFunction> currentFreeSlotFunction = freeSlotFunction; - if (currentFreeSlotConsumer != null) { - currentFreeSlotConsumer.accept(Tuple2.of(allocationId, cause)); + if (currentFreeSlotFunction != null) { + return currentFreeSlotFunction.apply(allocationId, cause); + } else { + return CompletableFuture.completedFuture(Acknowledge.get()); } - - return CompletableFuture.completedFuture(Acknowledge.get()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java index c3819747595b8..502b076e5714a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.clusterframework.types.SlotProfile; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.instance.SlotSharingGroupId; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -38,6 +39,7 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.taskexecutor.TaskExecutor; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; @@ -58,6 +60,7 @@ import javax.annotation.Nullable; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -67,6 +70,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import static org.apache.flink.runtime.jobmaster.slotpool.AvailableSlotsTest.DEFAULT_TESTING_PROFILE; import static org.hamcrest.MatcherAssert.assertThat; @@ -559,9 +563,15 @@ public void testShutdownReleasesAllSlots() throws Exception { final ArrayBlockingQueue freedSlotQueue = new ArrayBlockingQueue<>(numSlotOffers); - taskManagerGateway.setFreeSlotConsumer(tuple -> { - while(!freedSlotQueue.offer(tuple.f0)) {} - }); + taskManagerGateway.setFreeSlotFunction( + (AllocationID allocationID, Throwable cause) -> { + try { + freedSlotQueue.put(allocationID); + return CompletableFuture.completedFuture(Acknowledge.get()); + } catch (InterruptedException e) { + return FutureUtils.completedExceptionally(e); + } + }); final CompletableFuture> acceptedSlotOffersFuture = slotPoolGateway.offerSlots(taskManagerLocation, taskManagerGateway, slotOffers); @@ -598,7 +608,16 @@ public void testCheckIdleSlot() throws Exception { try { final BlockingQueue freedSlots = new ArrayBlockingQueue<>(1); - taskManagerGateway.setFreeSlotConsumer((tuple) -> freedSlots.offer(tuple.f0)); + taskManagerGateway.setFreeSlotFunction( + (AllocationID allocationId, Throwable cause) -> + { + try { + freedSlots.put(allocationId); + return CompletableFuture.completedFuture(Acknowledge.get()); + } catch (InterruptedException e) { + return FutureUtils.completedExceptionally(e); + } + }); final SlotPoolGateway slotPoolGateway = setupSlotPool(slotPool, resourceManagerGateway); @@ -634,6 +653,99 @@ public void testCheckIdleSlot() throws Exception { } } + /** + * Tests that idle slots which cannot be released are only recycled if the owning {@link TaskExecutor} + * is still registered at the {@link SlotPool}. See FLINK-9047. + */ + @Test + public void testReleasingIdleSlotFailed() throws Exception { + final ManualClock clock = new ManualClock(); + final SlotPool slotPool = new SlotPool( + rpcService, + jobId, + clock, + TestingUtils.infiniteTime(), + timeout); + + try { + final SlotPoolGateway slotPoolGateway = setupSlotPool(slotPool, resourceManagerGateway); + + final AllocationID expiredAllocationId = new AllocationID(); + final SlotOffer slotToExpire = new SlotOffer(expiredAllocationId, 0, ResourceProfile.UNKNOWN); + + final ArrayDeque> responseQueue = new ArrayDeque<>(2); + taskManagerGateway.setFreeSlotFunction((AllocationID allocationId, Throwable cause) -> { + if (responseQueue.isEmpty()) { + return CompletableFuture.completedFuture(Acknowledge.get()); + } else { + return responseQueue.pop(); + } + }); + + responseQueue.add(FutureUtils.completedExceptionally(new FlinkException("Test failure"))); + + final CompletableFuture responseFuture = new CompletableFuture<>(); + responseQueue.add(responseFuture); + + assertThat( + slotPoolGateway.registerTaskManager(taskManagerLocation.getResourceID()).get(), + Matchers.is(Acknowledge.get())); + + assertThat( + slotPoolGateway.offerSlot(taskManagerLocation, taskManagerGateway, slotToExpire).get(), + Matchers.is(true)); + + clock.advanceTime(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + slotPool.triggerCheckIdleSlot(); + + CompletableFuture allocatedSlotFuture = slotPoolGateway.allocateSlot( + new SlotRequestId(), + new DummyScheduledUnit(), + SlotProfile.noRequirements(), + true, + timeout); + + // wait until the slot has been fulfilled with the previously idling slot + final LogicalSlot logicalSlot = allocatedSlotFuture.get(); + assertThat(logicalSlot.getAllocationId(), Matchers.is(expiredAllocationId)); + + // return the slot + slotPool.getSlotOwner().returnAllocatedSlot(logicalSlot).get(); + + // advance the time so that the returned slot is now idling + clock.advanceTime(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + slotPool.triggerCheckIdleSlot(); + + // request a new slot after the idling slot has been released + allocatedSlotFuture = slotPoolGateway.allocateSlot( + new SlotRequestId(), + new DummyScheduledUnit(), + SlotProfile.noRequirements(), + true, + timeout); + + // release the TaskExecutor before we get a response from the slot releasing + slotPoolGateway.releaseTaskManager(taskManagerLocation.getResourceID()).get(); + + // let the slot releasing fail --> since the owning TaskExecutor is no longer registered + // the slot should be discarded + responseFuture.completeExceptionally(new FlinkException("Second test exception")); + + try { + // since the slot must have been discarded, we cannot fulfill the slot request + allocatedSlotFuture.get(10L, TimeUnit.MILLISECONDS); + fail("Expected to fail with a timeout."); + } catch (TimeoutException ignored) { + // expected + } + + } finally { + RpcUtils.terminateRpcEndpoint(slotPool, timeout); + } + } + private static SlotPoolGateway setupSlotPool( SlotPool slotPool, ResourceManagerGateway resourceManagerGateway) throws Exception { From 23b44f811b2dcfcdce8e0e2356f6c6c8226b73b2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Mar 2018 18:44:25 +0100 Subject: [PATCH 194/268] [hotfix] Remove unused method from SlotPool --- .../flink/runtime/jobmaster/slotpool/SlotPool.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java index 6040b41833695..662e71a7136c0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPool.java @@ -69,7 +69,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -724,15 +723,6 @@ private void slotRequestToResourceManagerFailed(SlotRequestId slotRequestID, Thr } } - private void checkTimeoutSlotAllocation(SlotRequestId slotRequestID) { - PendingRequest request = pendingRequests.removeKeyA(slotRequestID); - if (request != null) { - failPendingRequest( - request, - new TimeoutException("Slot allocation request " + slotRequestID + " timed out")); - } - } - private void stashRequestWaitingForResourceManager(final PendingRequest pendingRequest) { log.info("Cannot serve slot request, no ResourceManager connected. " + From 5f72ca57d817fbd77e8621cab3e21e1e57c34708 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Mar 2018 21:08:13 +0100 Subject: [PATCH 195/268] [hotfix] Make RestServerEndpoint#uploadDir protected --- .../flink/runtime/dispatcher/DispatcherRestEndpoint.java | 4 ---- .../org/apache/flink/runtime/rest/RestServerEndpoint.java | 3 ++- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java index 45185528395d3..8072cf45a7100 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java @@ -42,7 +42,6 @@ import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandler; import java.io.IOException; -import java.nio.file.Path; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -52,8 +51,6 @@ */ public class DispatcherRestEndpoint extends WebMonitorEndpoint { - private final Path uploadDir; - private WebMonitorExtension webSubmissionExtension; public DispatcherRestEndpoint( @@ -80,7 +77,6 @@ public DispatcherRestEndpoint( leaderElectionService, fatalErrorHandler); - uploadDir = endpointConfiguration.getUploadDir(); webSubmissionExtension = WebMonitorExtension.empty(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index dfb01ca2657c9..80d9140d041b9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -77,8 +77,9 @@ public abstract class RestServerEndpoint { private final String restBindAddress; private final int restBindPort; private final SSLEngine sslEngine; - private final Path uploadDir; private final int maxContentLength; + + protected final Path uploadDir; protected final Map responseHeaders; private final CompletableFuture terminationFuture; From 4fa4e8cb364c35ea1807a051929b4604b9d31c2e Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Mar 2018 21:04:40 +0100 Subject: [PATCH 196/268] [FLINK-9027] [web] Clean up web UI resources by installing shut down hook The ClusterEntrypoint creates temp directory for the RestServerEndpoint. This directory contains the web ui files and if not differently configured the web upload directory. In case of a hard shut down, as it happens with bin/stop-cluster.sh the ClusterEntrypoint will clean up this directory by installing a shut down hook. All future directory cleanup tasks should go into this method ClusterEntrypoin#cleanupDirectories. This closes #5740. --- .../program/rest/RestClusterClientTest.java | 5 -- .../runtime/entrypoint/ClusterEntrypoint.java | 55 +++++++++++++++++-- .../runtime/minicluster/MiniCluster.java | 2 +- .../runtime/rest/RestServerEndpoint.java | 14 ++--- .../rest/RestServerEndpointConfiguration.java | 3 +- .../handler/RestHandlerConfiguration.java | 20 +++---- .../webmonitor/WebMonitorEndpoint.java | 11 ++-- .../rest/RestServerEndpointITCase.java | 2 +- 8 files changed, 74 insertions(+), 38 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index e108a0b116eb3..e98ba436abf29 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -666,11 +666,6 @@ private class TestRestServerEndpoint extends RestServerEndpoint implements AutoC @Override protected void startInternal() throws Exception {} - - @Override - public void close() throws Exception { - shutDownAsync().get(); - } } @FunctionalInterface diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 63c8072a82620..8a4db0367bfe1 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -69,7 +69,9 @@ import org.apache.flink.runtime.webmonitor.retriever.impl.AkkaQueryServiceRetriever; import org.apache.flink.runtime.webmonitor.retriever.impl.RpcGatewayRetriever; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FileUtils; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ShutdownHookUtil; import akka.actor.ActorSystem; import org.slf4j.Logger; @@ -78,10 +80,14 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; +import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; +import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -159,9 +165,13 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler { @GuardedBy("lock") private JobManagerMetricGroup jobManagerMetricGroup; + private final Thread shutDownHook; + protected ClusterEntrypoint(Configuration configuration) { - this.configuration = Preconditions.checkNotNull(configuration); + this.configuration = generateClusterConfiguration(configuration); this.terminationFuture = new CompletableFuture<>(); + + shutDownHook = ShutdownHookUtil.addShutdownHook(this::cleanupDirectories, getClass().getSimpleName(), LOG); } public CompletableFuture getTerminationFuture() { @@ -479,7 +489,7 @@ protected CompletableFuture stopClusterComponents() { } if (webMonitorEndpoint != null) { - terminationFutures.add(webMonitorEndpoint.shutDownAsync()); + terminationFutures.add(webMonitorEndpoint.closeAsync()); } if (dispatcher != null) { @@ -523,6 +533,17 @@ public void onFatalError(Throwable exception) { // Internal methods // -------------------------------------------------- + private Configuration generateClusterConfiguration(Configuration configuration) { + final Configuration resultConfiguration = new Configuration(Preconditions.checkNotNull(configuration)); + + final String webTmpDir = configuration.getString(WebOptions.TMP_DIR); + final Path uniqueWebTmpDir = Paths.get(webTmpDir, "flink-web-" + UUID.randomUUID()); + + resultConfiguration.setString(WebOptions.TMP_DIR, uniqueWebTmpDir.toAbsolutePath().toString()); + + return resultConfiguration; + } + private CompletableFuture shutDownAsync(boolean cleanupHaData) { if (isShutDown.compareAndSet(false, true)) { LOG.info("Stopping {}.", getClass().getSimpleName()); @@ -535,11 +556,22 @@ private CompletableFuture shutDownAsync(boolean cleanupHaData) { serviceShutdownFuture.whenComplete( (Void ignored2, Throwable serviceThrowable) -> { + Throwable finalException = null; + if (serviceThrowable != null) { - terminationFuture.completeExceptionally( - ExceptionUtils.firstOrSuppressed(serviceThrowable, componentThrowable)); + finalException = ExceptionUtils.firstOrSuppressed(serviceThrowable, componentThrowable); } else if (componentThrowable != null) { - terminationFuture.completeExceptionally(componentThrowable); + finalException = componentThrowable; + } + + try { + cleanupDirectories(); + } catch (IOException e) { + finalException = ExceptionUtils.firstOrSuppressed(e, finalException); + } + + if (finalException != null) { + terminationFuture.completeExceptionally(finalException); } else { terminationFuture.complete(null); } @@ -576,6 +608,19 @@ private void shutDownAndTerminate( } } + /** + * Clean up of temporary directories created by the {@link ClusterEntrypoint}. + * + * @throws IOException if the temporary directories could not be cleaned up + */ + private void cleanupDirectories() throws IOException { + ShutdownHookUtil.removeShutdownHook(shutDownHook, getClass().getSimpleName(), LOG); + + final String webTmpDir = configuration.getString(WebOptions.TMP_DIR); + + FileUtils.deleteDirectory(new File(webTmpDir)); + } + // -------------------------------------------------- // Abstract methods // -------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index bc75a547b55f5..0da6f333b6830 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -758,7 +758,7 @@ private CompletableFuture shutDownDispatcher() { } if (dispatcherRestEndpoint != null) { - terminationFutures.add(dispatcherRestEndpoint.shutDownAsync()); + terminationFutures.add(dispatcherRestEndpoint.closeAsync()); dispatcherRestEndpoint = null; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java index 80d9140d041b9..15fbbb24866ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpoint.java @@ -25,8 +25,8 @@ import org.apache.flink.runtime.rest.handler.PipelineErrorHandler; import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; import org.apache.flink.runtime.rest.handler.RouterHandler; +import org.apache.flink.util.AutoCloseableAsync; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FileUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap; @@ -67,7 +67,7 @@ /** * An abstract class for netty-based REST server endpoints. */ -public abstract class RestServerEndpoint { +public abstract class RestServerEndpoint implements AutoCloseableAsync { protected final Logger log = LoggerFactory.getLogger(getClass()); @@ -256,7 +256,8 @@ public String getRestBaseUrl() { } } - public final CompletableFuture shutDownAsync() { + @Override + public CompletableFuture closeAsync() { synchronized (lock) { log.info("Shutting down rest endpoint."); @@ -370,12 +371,7 @@ protected CompletableFuture shutDownInternal() { }); }); - return FutureUtils.runAfterwards( - channelTerminationFuture, - () -> { - log.info("Cleaning upload directory {}", uploadDir); - FileUtils.cleanDirectory(uploadDir.toFile()); - }); + return channelTerminationFuture; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java index 1fac08e53edc9..8af76f5bfd5c0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/RestServerEndpointConfiguration.java @@ -36,7 +36,6 @@ import java.nio.file.Paths; import java.util.Collections; import java.util.Map; -import java.util.UUID; import static java.util.Objects.requireNonNull; @@ -172,7 +171,7 @@ public static RestServerEndpointConfiguration fromConfiguration(Configuration co final Path uploadDir = Paths.get( config.getString(WebOptions.UPLOAD_DIR, config.getString(WebOptions.TMP_DIR)), - "flink-web-upload-" + UUID.randomUUID()); + "flink-web-upload"); final int maxContentLength = config.getInteger(RestOptions.REST_SERVER_MAX_CONTENT_LENGTH); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java index f92946bd0f5f4..3f6516aadde72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java @@ -23,8 +23,8 @@ import org.apache.flink.configuration.WebOptions; import org.apache.flink.util.Preconditions; -import java.io.File; -import java.util.UUID; +import java.nio.file.Path; +import java.nio.file.Paths; /** * Configuration object containing values for the rest handler configuration. @@ -37,20 +37,20 @@ public class RestHandlerConfiguration { private final Time timeout; - private final File tmpDir; + private final Path webUiDir; public RestHandlerConfiguration( long refreshInterval, int maxCheckpointStatisticCacheEntries, Time timeout, - File tmpDir) { + Path webUiDir) { Preconditions.checkArgument(refreshInterval > 0L, "The refresh interval (ms) should be larger than 0."); this.refreshInterval = refreshInterval; this.maxCheckpointStatisticCacheEntries = maxCheckpointStatisticCacheEntries; this.timeout = Preconditions.checkNotNull(timeout); - this.tmpDir = Preconditions.checkNotNull(tmpDir); + this.webUiDir = Preconditions.checkNotNull(webUiDir); } public long getRefreshInterval() { @@ -65,8 +65,8 @@ public Time getTimeout() { return timeout; } - public File getTmpDir() { - return tmpDir; + public Path getWebUiDir() { + return webUiDir; } public static RestHandlerConfiguration fromConfiguration(Configuration configuration) { @@ -76,13 +76,13 @@ public static RestHandlerConfiguration fromConfiguration(Configuration configura final Time timeout = Time.milliseconds(configuration.getLong(WebOptions.TIMEOUT)); - final String rootDir = "flink-web-" + UUID.randomUUID(); - final File tmpDir = new File(configuration.getString(WebOptions.TMP_DIR), rootDir); + final String rootDir = "flink-web-ui"; + final Path webUiDir = Paths.get(configuration.getString(WebOptions.TMP_DIR), rootDir); return new RestHandlerConfiguration( refreshInterval, maxCheckpointStatisticCacheEntries, timeout, - tmpDir); + webUiDir); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index 50ad7eb1bceeb..d4aa94e19feaa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -127,6 +127,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -498,7 +499,7 @@ protected List> initiali executor, metricFetcher); - final File tmpDir = restConfiguration.getTmpDir(); + final Path webUiDir = restConfiguration.getWebUiDir(); Optional> optWebContent; @@ -507,7 +508,7 @@ protected List> initiali leaderRetriever, restAddressFuture, timeout, - tmpDir); + webUiDir.toFile()); } catch (IOException e) { log.warn("Could not load web content handler.", e); optWebContent = Optional.empty(); @@ -635,15 +636,15 @@ protected CompletableFuture shutDownInternal() { final CompletableFuture shutdownFuture = super.shutDownInternal(); - final File tmpDir = restConfiguration.getTmpDir(); + final Path webUiDir = restConfiguration.getWebUiDir(); return FutureUtils.runAfterwardsAsync( shutdownFuture, () -> { Exception exception = null; try { - log.info("Removing cache directory {}", tmpDir); - FileUtils.deleteDirectory(tmpDir); + log.info("Removing cache directory {}", webUiDir); + FileUtils.deleteDirectory(webUiDir.toFile()); } catch (Exception e) { exception = e; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java index 784c14158a3b4..e510798069e9d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java @@ -154,7 +154,7 @@ public void teardown() throws Exception { } if (serverEndpoint != null) { - serverEndpoint.shutDownAsync().get(); + serverEndpoint.close(); serverEndpoint = null; } } From 1d96d51d061922ce90d53690e60f58a0dbf429ab Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Mar 2018 21:48:19 +0100 Subject: [PATCH 197/268] [hotfix] Log final status and exit code under lock --- .../flink/runtime/entrypoint/ClusterEntrypoint.java | 10 +++++----- .../flink/runtime/entrypoint/JobClusterEntrypoint.java | 3 +-- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 8a4db0367bfe1..50d0db335f442 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -587,12 +587,12 @@ private void shutDownAndTerminate( ApplicationStatus applicationStatus, boolean cleanupHaData) { - LOG.info("Shut down and terminate {} with return code {} and application status {}.", - getClass().getSimpleName(), - returnCode, - applicationStatus); - if (isTerminating.compareAndSet(false, true)) { + LOG.info("Shut down and terminate {} with return code {} and application status {}.", + getClass().getSimpleName(), + returnCode, + applicationStatus); + shutDownAsync(cleanupHaData).whenComplete( (Void ignored, Throwable t) -> { if (t != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java index df950a343be94..95d9c742bef4a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/JobClusterEntrypoint.java @@ -44,7 +44,6 @@ import javax.annotation.Nullable; -import java.io.IOException; import java.util.concurrent.Executor; /** @@ -83,7 +82,7 @@ protected MiniDispatcherRestEndpoint createRestEndpoint( @Override protected ArchivedExecutionGraphStore createSerializableExecutionGraphStore( Configuration configuration, - ScheduledExecutor scheduledExecutor) throws IOException { + ScheduledExecutor scheduledExecutor) { return new MemoryArchivedExecutionGraphStore(); } From 2afb70de4a790b90723dd7f41eb934e4a8fb468f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Mar 2018 22:14:58 +0100 Subject: [PATCH 198/268] [hotfix] Add FutureUtils#composeAfterwards --- .../flink/runtime/concurrent/FutureUtils.java | 34 +++++ .../runtime/concurrent/FutureUtilsTest.java | 119 ++++++++++++++++++ 2 files changed, 153 insertions(+) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java index e0164a92f6a99..51740e3a1aba5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/concurrent/FutureUtils.java @@ -411,6 +411,40 @@ public static CompletableFuture runAfterwardsAsync( return resultFuture; } + /** + * Run the given asynchronous action after the completion of the given future. The given future can be + * completed normally or exceptionally. In case of an exceptional completion, the + * asynchronous action's exception will be added to the initial exception. + * + * @param future to wait for its completion + * @param composedAction asynchronous action which is triggered after the future's completion + * @return Future which is completed after the asynchronous action has completed. This future can contain + * an exception if an error occurred in the given future or asynchronous action. + */ + public static CompletableFuture composeAfterwards( + CompletableFuture future, + Supplier> composedAction) { + final CompletableFuture resultFuture = new CompletableFuture<>(); + + future.whenComplete( + (Object outerIgnored, Throwable outerThrowable) -> { + final CompletableFuture composedActionFuture = composedAction.get(); + + composedActionFuture.whenComplete( + (Object innerIgnored, Throwable innerThrowable) -> { + if (innerThrowable != null) { + resultFuture.completeExceptionally(ExceptionUtils.firstOrSuppressed(innerThrowable, outerThrowable)); + } else if (outerThrowable != null) { + resultFuture.completeExceptionally(outerThrowable); + } else { + resultFuture.complete(null); + } + }); + }); + + return resultFuture; + } + // ------------------------------------------------------------------------ // composing futures // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java index 57f6bd01029ab..df2a0c748c343 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java @@ -46,6 +46,7 @@ import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.Matchers.arrayContaining; +import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.emptyArray; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; @@ -327,6 +328,124 @@ public void testRunAfterwardsExceptional() throws Exception { } } + @Test + public void testComposeAfterwards() throws ExecutionException, InterruptedException { + final CompletableFuture inputFuture = new CompletableFuture<>(); + final OneShotLatch composeLatch = new OneShotLatch(); + + final CompletableFuture composeFuture = FutureUtils.composeAfterwards( + inputFuture, + () -> { + composeLatch.trigger(); + return CompletableFuture.completedFuture(null); + }); + + assertThat(composeLatch.isTriggered(), is(false)); + assertThat(composeFuture.isDone(), is(false)); + + inputFuture.complete(null); + + assertThat(composeLatch.isTriggered(), is(true)); + assertThat(composeFuture.isDone(), is(true)); + + // check that tthis future is not exceptionally completed + composeFuture.get(); + } + + @Test + public void testComposeAfterwardsFirstExceptional() throws InterruptedException { + final CompletableFuture inputFuture = new CompletableFuture<>(); + final OneShotLatch composeLatch = new OneShotLatch(); + final FlinkException testException = new FlinkException("Test exception"); + + final CompletableFuture composeFuture = FutureUtils.composeAfterwards( + inputFuture, + () -> { + composeLatch.trigger(); + return CompletableFuture.completedFuture(null); + }); + + assertThat(composeLatch.isTriggered(), is(false)); + assertThat(composeFuture.isDone(), is(false)); + + inputFuture.completeExceptionally(testException); + + assertThat(composeLatch.isTriggered(), is(true)); + assertThat(composeFuture.isDone(), is(true)); + + // check that this future is not exceptionally completed + try { + composeFuture.get(); + fail("Expected an exceptional completion"); + } catch (ExecutionException ee) { + assertThat(ExceptionUtils.stripExecutionException(ee), is(testException)); + } + } + + @Test + public void testComposeAfterwardsSecondExceptional() throws InterruptedException { + final CompletableFuture inputFuture = new CompletableFuture<>(); + final OneShotLatch composeLatch = new OneShotLatch(); + final FlinkException testException = new FlinkException("Test exception"); + + final CompletableFuture composeFuture = FutureUtils.composeAfterwards( + inputFuture, + () -> { + composeLatch.trigger(); + return FutureUtils.completedExceptionally(testException); + }); + + assertThat(composeLatch.isTriggered(), is(false)); + assertThat(composeFuture.isDone(), is(false)); + + inputFuture.complete(null); + + assertThat(composeLatch.isTriggered(), is(true)); + assertThat(composeFuture.isDone(), is(true)); + + // check that this future is not exceptionally completed + try { + composeFuture.get(); + fail("Expected an exceptional completion"); + } catch (ExecutionException ee) { + assertThat(ExceptionUtils.stripExecutionException(ee), is(testException)); + } + } + + @Test + public void testComposeAfterwardsBothExceptional() throws InterruptedException { + final CompletableFuture inputFuture = new CompletableFuture<>(); + final FlinkException testException1 = new FlinkException("Test exception1"); + final FlinkException testException2 = new FlinkException("Test exception2"); + final OneShotLatch composeLatch = new OneShotLatch(); + + final CompletableFuture composeFuture = FutureUtils.composeAfterwards( + inputFuture, + () -> { + composeLatch.trigger(); + return FutureUtils.completedExceptionally(testException2); + }); + + assertThat(composeLatch.isTriggered(), is(false)); + assertThat(composeFuture.isDone(), is(false)); + + inputFuture.completeExceptionally(testException1); + + assertThat(composeLatch.isTriggered(), is(true)); + assertThat(composeFuture.isDone(), is(true)); + + // check that this future is not exceptionally completed + try { + composeFuture.get(); + fail("Expected an exceptional completion"); + } catch (ExecutionException ee) { + final Throwable actual = ExceptionUtils.stripExecutionException(ee); + assertThat(actual, is(testException1)); + assertThat(actual.getSuppressed(), arrayWithSize(1)); + assertThat(actual.getSuppressed()[0], is(testException2)); + } + } + @Test public void testCompleteAll() throws Exception { final CompletableFuture inputFuture1 = new CompletableFuture<>(); From c94b88aa401f4561a841aa9a986a8eeb692c7ccf Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 21 Mar 2018 22:19:28 +0100 Subject: [PATCH 199/268] [FLINK-8900] [yarn] Properly unregister application from Yarn RM This closes #5741. --- .../MesosResourceManager.java | 4 +- .../MesosResourceManagerTest.java | 2 +- .../runtime/entrypoint/ClusterEntrypoint.java | 81 ++++++++++++------- .../resourcemanager/ResourceManager.java | 16 ++-- .../ResourceManagerGateway.java | 11 ++- .../StandaloneResourceManager.java | 2 +- .../TestingResourceManager.java | 2 +- .../utils/TestingResourceManagerGateway.java | 4 +- .../flink/yarn/YarnResourceManager.java | 6 +- 9 files changed, 77 insertions(+), 51 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java index 1f58b119f2768..4f4a6d1942c3a 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManager.java @@ -362,9 +362,9 @@ public CompletableFuture postStop() { } @Override - protected void shutDownApplication( + protected void internalDeregisterApplication( ApplicationStatus finalStatus, - @Nullable String optionalDiagnostics) throws ResourceManagerException { + @Nullable String diagnostics) throws ResourceManagerException { LOG.info("Shutting down and unregistering as a Mesos framework."); Exception exception = null; diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java index 412e18da65a30..5d9a6cffd7038 100644 --- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosResourceManagerTest.java @@ -737,7 +737,7 @@ public void testStopWorker() throws Exception { public void testShutdownApplication() throws Exception { new Context() {{ startResourceManager(); - resourceManager.shutDownCluster(ApplicationStatus.SUCCEEDED, ""); + resourceManager.deregisterApplication(ApplicationStatus.SUCCEEDED, ""); // verify that the Mesos framework is shutdown verify(rmServices.schedulerDriver).stop(false); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java index 50d0db335f442..b25729bd1e023 100755 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java @@ -197,6 +197,7 @@ protected void startCluster() { shutDownAndTerminate( STARTUP_FAILURE_RETURN_CODE, ApplicationStatus.FAILED, + t.getMessage(), false); } } @@ -245,6 +246,7 @@ protected void runCluster(Configuration configuration) throws Exception { shutDownAndTerminate( SUCCESS_RETURN_CODE, ApplicationStatus.SUCCEEDED, + throwable != null ? throwable.getMessage() : null, true); }); } @@ -544,38 +546,34 @@ private Configuration generateClusterConfiguration(Configuration configuration) return resultConfiguration; } - private CompletableFuture shutDownAsync(boolean cleanupHaData) { + private CompletableFuture shutDownAsync( + boolean cleanupHaData, + ApplicationStatus applicationStatus, + @Nullable String diagnostics) { if (isShutDown.compareAndSet(false, true)) { LOG.info("Stopping {}.", getClass().getSimpleName()); - final CompletableFuture componentShutdownFuture = stopClusterComponents(); - - componentShutdownFuture.whenComplete( - (Void ignored1, Throwable componentThrowable) -> { - final CompletableFuture serviceShutdownFuture = stopClusterServices(cleanupHaData); - - serviceShutdownFuture.whenComplete( - (Void ignored2, Throwable serviceThrowable) -> { - Throwable finalException = null; - - if (serviceThrowable != null) { - finalException = ExceptionUtils.firstOrSuppressed(serviceThrowable, componentThrowable); - } else if (componentThrowable != null) { - finalException = componentThrowable; - } - - try { - cleanupDirectories(); - } catch (IOException e) { - finalException = ExceptionUtils.firstOrSuppressed(e, finalException); - } - - if (finalException != null) { - terminationFuture.completeExceptionally(finalException); - } else { - terminationFuture.complete(null); - } - }); + final CompletableFuture shutDownApplicationFuture = deregisterApplication(applicationStatus, diagnostics); + + final CompletableFuture componentShutdownFuture = FutureUtils.composeAfterwards( + shutDownApplicationFuture, + this::stopClusterComponents); + + final CompletableFuture serviceShutdownFuture = FutureUtils.composeAfterwards( + componentShutdownFuture, + () -> stopClusterServices(cleanupHaData)); + + final CompletableFuture cleanupDirectoriesFuture = FutureUtils.runAfterwards( + serviceShutdownFuture, + this::cleanupDirectories); + + cleanupDirectoriesFuture.whenComplete( + (Void ignored2, Throwable serviceThrowable) -> { + if (serviceThrowable != null) { + terminationFuture.completeExceptionally(serviceThrowable); + } else { + terminationFuture.complete(null); + } }); } @@ -585,6 +583,7 @@ private CompletableFuture shutDownAsync(boolean cleanupHaData) { private void shutDownAndTerminate( int returnCode, ApplicationStatus applicationStatus, + @Nullable String diagnostics, boolean cleanupHaData) { if (isTerminating.compareAndSet(false, true)) { @@ -593,7 +592,10 @@ private void shutDownAndTerminate( returnCode, applicationStatus); - shutDownAsync(cleanupHaData).whenComplete( + shutDownAsync( + cleanupHaData, + applicationStatus, + diagnostics).whenComplete( (Void ignored, Throwable t) -> { if (t != null) { LOG.info("Could not properly shut down cluster entrypoint.", t); @@ -608,6 +610,25 @@ private void shutDownAndTerminate( } } + /** + * Deregister the Flink application from the resource management system by signalling + * the {@link ResourceManager}. + * + * @param applicationStatus to terminate the application with + * @param diagnostics additional information about the shut down, can be {@code null} + * @return Future which is completed once the shut down + */ + private CompletableFuture deregisterApplication(ApplicationStatus applicationStatus, @Nullable String diagnostics) { + synchronized (lock) { + if (resourceManager != null) { + final ResourceManagerGateway selfGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); + return selfGateway.deregisterApplication(applicationStatus, diagnostics).thenApply(ack -> null); + } else { + return CompletableFuture.completedFuture(null); + } + } + } + /** * Clean up of temporary directories created by the {@link ClusterEntrypoint}. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java index cae9c6cdb383c..c75346900cdd8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java @@ -486,19 +486,21 @@ public void unRegisterInfoMessageListener(final String address) { * Cleanup application and shut down cluster. * * @param finalStatus of the Flink application - * @param optionalDiagnostics diagnostics message for the Flink application or {@code null} + * @param diagnostics diagnostics message for the Flink application or {@code null} */ @Override - public void shutDownCluster( + public CompletableFuture deregisterApplication( final ApplicationStatus finalStatus, - @Nullable final String optionalDiagnostics) { - log.info("Shut down cluster because application is in {}, diagnostics {}.", finalStatus, optionalDiagnostics); + @Nullable final String diagnostics) { + log.info("Shut down cluster because application is in {}, diagnostics {}.", finalStatus, diagnostics); try { - shutDownApplication(finalStatus, optionalDiagnostics); + internalDeregisterApplication(finalStatus, diagnostics); } catch (ResourceManagerException e) { log.warn("Could not properly shutdown the application.", e); } + + return CompletableFuture.completedFuture(Acknowledge.get()); } @Override @@ -946,7 +948,7 @@ public void handleError(final Exception exception) { protected abstract void initialize() throws ResourceManagerException; /** - * The framework specific code for shutting down the application. This should report the + * The framework specific code to deregister the application. This should report the * application's final status and shut down the resource manager cleanly. * *

    This method also needs to make sure all pending containers that are not registered @@ -956,7 +958,7 @@ public void handleError(final Exception exception) { * @param optionalDiagnostics A diagnostics message or {@code null}. * @throws ResourceManagerException if the application could not be shut down. */ - protected abstract void shutDownApplication( + protected abstract void internalDeregisterApplication( ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws ResourceManagerException; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java index 836bc0b0faf00..bd282d6cff0bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java @@ -41,6 +41,8 @@ import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutor; +import javax.annotation.Nullable; + import java.util.Collection; import java.util.concurrent.CompletableFuture; @@ -133,11 +135,12 @@ void notifySlotAvailable( void unRegisterInfoMessageListener(String infoMessageListenerAddress); /** - * shutdown cluster - * @param finalStatus - * @param optionalDiagnostics + * Deregister Flink from the underlying resource management system. + * + * @param finalStatus final status with which to deregister the Flink application + * @param diagnostics additional information for the resource management system, can be {@code null} */ - void shutDownCluster(final ApplicationStatus finalStatus, final String optionalDiagnostics); + CompletableFuture deregisterApplication(final ApplicationStatus finalStatus, @Nullable final String diagnostics); /** * Gets the currently registered number of TaskManagers. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java index 7226d296f373d..d8e0e480a2a4d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/StandaloneResourceManager.java @@ -72,7 +72,7 @@ protected void initialize() throws ResourceManagerException { } @Override - protected void shutDownApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) { + protected void internalDeregisterApplication(ApplicationStatus finalStatus, @Nullable String diagnostics) { } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java index 3db9be032e6ea..2bd976bd9bf32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/TestingResourceManager.java @@ -68,7 +68,7 @@ protected void initialize() throws ResourceManagerException { } @Override - protected void shutDownApplication(ApplicationStatus finalStatus, @Nullable String optionalDiagnostics) throws ResourceManagerException { + protected void internalDeregisterApplication(ApplicationStatus finalStatus, @Nullable String diagnostics) throws ResourceManagerException { // noop } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java index 33c6c08d66704..9b4041414d62e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/TestingResourceManagerGateway.java @@ -214,8 +214,8 @@ public void unRegisterInfoMessageListener(String infoMessageListenerAddress) { } @Override - public void shutDownCluster(ApplicationStatus finalStatus, String optionalDiagnostics) { - + public CompletableFuture deregisterApplication(ApplicationStatus finalStatus, String diagnostics) { + return CompletableFuture.completedFuture(Acknowledge.get()); } @Override diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java index 97db2ad8a37a1..bfe7d65262ac6 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnResourceManager.java @@ -266,16 +266,16 @@ public CompletableFuture postStop() { } @Override - protected void shutDownApplication( + protected void internalDeregisterApplication( ApplicationStatus finalStatus, - @Nullable String optionalDiagnostics) { + @Nullable String diagnostics) { // first, de-register from YARN FinalApplicationStatus yarnStatus = getYarnStatus(finalStatus); log.info("Unregister application from the YARN Resource Manager with final status {}.", yarnStatus); try { - resourceManagerClient.unregisterApplicationMaster(yarnStatus, optionalDiagnostics, ""); + resourceManagerClient.unregisterApplicationMaster(yarnStatus, diagnostics, ""); } catch (Throwable t) { log.error("Could not unregister the application master.", t); } From 0acc1e299fe34f9562c52e062d9759b7afe46dcc Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 19 Mar 2018 11:36:39 +0100 Subject: [PATCH 200/268] [FLINK-8956][tests] Port RescalingITCase to flip6 This closes #5715. --- .../test/checkpointing/RescalingITCase.java | 282 ++++++------------ 1 file changed, 97 insertions(+), 185 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index a23c679e65f2b..e4f4389bb6abc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -25,23 +25,24 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; -import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; -import org.apache.flink.runtime.testingUtils.TestingCluster; -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; +import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.datastream.DataStream; @@ -50,7 +51,9 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.util.Collector; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; @@ -62,25 +65,20 @@ import org.junit.runners.Parameterized; import java.io.File; +import java.time.Duration; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import scala.Option; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; +import java.util.stream.Collectors; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * Test savepoint rescaling. @@ -106,7 +104,7 @@ enum OperatorCheckpointMethod { NON_PARTITIONED, CHECKPOINTED_FUNCTION, CHECKPOINTED_FUNCTION_BROADCAST, LIST_CHECKPOINTED } - private static TestingCluster cluster; + private static MiniClusterResource cluster; @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -120,8 +118,6 @@ public void setup() throws Exception { currentBackend = backend; Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, slotsPerTaskManager); final File checkpointDir = temporaryFolder.newFolder(); final File savepointDir = temporaryFolder.newFolder(); @@ -130,15 +126,20 @@ public void setup() throws Exception { config.setString(CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); config.setString(CheckpointingOptions.SAVEPOINT_DIRECTORY, savepointDir.toURI().toString()); - cluster = new TestingCluster(config); - cluster.start(); + cluster = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + config, + numTaskManagers, + slotsPerTaskManager), + true); + cluster.before(); } } @AfterClass public static void shutDownExistingCluster() { if (cluster != null) { - cluster.stop(); + cluster.after(); cluster = null; } } @@ -175,20 +176,18 @@ public void testSavepointRescalingKeyedState(boolean scaleOut, boolean deriveMax final int parallelism2 = scaleOut ? numSlots : numSlots / 2; final int maxParallelism = 13; - FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES); - Deadline deadline = timeout.fromNow(); + Duration timeout = Duration.ofMinutes(3); + Deadline deadline = Deadline.now().plus(timeout); - ActorGateway jobManager = null; - JobID jobID = null; + ClusterClient client = cluster.getClusterClient(); try { - jobManager = cluster.getLeaderGateway(deadline.timeLeft()); - JobGraph jobGraph = createJobGraphWithKeyedState(parallelism, maxParallelism, numberKeys, numberElements, false, 100); - jobID = jobGraph.getJobID(); + final JobID jobID = jobGraph.getJobID(); - cluster.submitJobDetached(jobGraph); + client.setDetached(true); + client.submitJob(jobGraph, RescalingITCase.class.getClassLoader()); // wait til the sources have emitted numberElements for each key and completed a checkpoint SubtaskIndexFlatMapper.workCompletedLatch.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); @@ -210,22 +209,15 @@ public void testSavepointRescalingKeyedState(boolean scaleOut, boolean deriveMax // clear the CollectionSink set for the restarted job CollectionSink.clearElementsSet(); - Future savepointPathFuture = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID, Option.empty()), deadline.timeLeft()); - - final String savepointPath = ((JobManagerMessages.TriggerSavepointSuccess) - Await.result(savepointPathFuture, deadline.timeLeft())).savepointPath(); - - Future jobRemovedFuture = jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), deadline.timeLeft()); + CompletableFuture savepointPathFuture = client.triggerSavepoint(jobID, null); - Future cancellationResponseFuture = jobManager.ask(new JobManagerMessages.CancelJob(jobID), deadline.timeLeft()); + final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - Object cancellationResponse = Await.result(cancellationResponseFuture, deadline.timeLeft()); + client.cancel(jobID); - assertTrue(cancellationResponse instanceof JobManagerMessages.CancellationSuccess); - - Await.ready(jobRemovedFuture, deadline.timeLeft()); - - jobID = null; + while (!getRunningJobs(client).isEmpty()) { + Thread.sleep(50); + } int restoreMaxParallelism = deriveMaxParallelism ? ExecutionJobVertex.VALUE_NOT_SET : maxParallelism; @@ -233,11 +225,8 @@ public void testSavepointRescalingKeyedState(boolean scaleOut, boolean deriveMax scaledJobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - jobID = scaledJobGraph.getJobID(); - - cluster.submitJobAndWait(scaledJobGraph, false); - - jobID = null; + client.setDetached(false); + client.submitJob(scaledJobGraph, RescalingITCase.class.getClassLoader()); Set> actualResult2 = CollectionSink.getElementsSet(); @@ -253,17 +242,6 @@ public void testSavepointRescalingKeyedState(boolean scaleOut, boolean deriveMax } finally { // clear the CollectionSink set for the restarted job CollectionSink.clearElementsSet(); - - // clear any left overs from a possibly failed job - if (jobID != null && jobManager != null) { - Future jobRemovedFuture = jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), timeout); - - try { - Await.ready(jobRemovedFuture, timeout); - } catch (TimeoutException | InterruptedException ie) { - fail("Failed while cleaning up the cluster."); - } - } } } @@ -279,57 +257,39 @@ public void testSavepointRescalingNonPartitionedStateCausesException() throws Ex final int parallelism2 = numSlots; final int maxParallelism = 13; - FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES); - Deadline deadline = timeout.fromNow(); + Duration timeout = Duration.ofMinutes(3); + Deadline deadline = Deadline.now().plus(timeout); - JobID jobID = null; - ActorGateway jobManager = null; + ClusterClient client = cluster.getClusterClient(); try { - jobManager = cluster.getLeaderGateway(deadline.timeLeft()); - JobGraph jobGraph = createJobGraphWithOperatorState(parallelism, maxParallelism, OperatorCheckpointMethod.NON_PARTITIONED); - jobID = jobGraph.getJobID(); - - cluster.submitJobDetached(jobGraph); + final JobID jobID = jobGraph.getJobID(); - Object savepointResponse = null; + client.setDetached(true); + client.submitJob(jobGraph, RescalingITCase.class.getClassLoader()); // wait until the operator is started StateSourceBase.workStartedLatch.await(); - Future savepointPathFuture = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID, Option.empty()), deadline.timeLeft()); - FiniteDuration waitingTime = new FiniteDuration(10, TimeUnit.SECONDS); - savepointResponse = Await.result(savepointPathFuture, waitingTime); + CompletableFuture savepointPathFuture = client.triggerSavepoint(jobID, null); - assertTrue(String.valueOf(savepointResponse), savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess); + final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - final String savepointPath = ((JobManagerMessages.TriggerSavepointSuccess) savepointResponse).savepointPath(); + client.cancel(jobID); - Future jobRemovedFuture = jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), deadline.timeLeft()); - - Future cancellationResponseFuture = jobManager.ask(new JobManagerMessages.CancelJob(jobID), deadline.timeLeft()); - - Object cancellationResponse = Await.result(cancellationResponseFuture, deadline.timeLeft()); - - assertTrue(cancellationResponse instanceof JobManagerMessages.CancellationSuccess); - - Await.ready(jobRemovedFuture, deadline.timeLeft()); + while (!getRunningJobs(client).isEmpty()) { + Thread.sleep(50); + } // job successfully removed - jobID = null; - JobGraph scaledJobGraph = createJobGraphWithOperatorState(parallelism2, maxParallelism, OperatorCheckpointMethod.NON_PARTITIONED); scaledJobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - jobID = scaledJobGraph.getJobID(); - - cluster.submitJobAndWait(scaledJobGraph, false); - - jobID = null; - + client.setDetached(false); + client.submitJob(scaledJobGraph, RescalingITCase.class.getClassLoader()); } catch (JobExecutionException exception) { if (exception.getCause() instanceof IllegalStateException) { // we expect a IllegalStateException wrapped @@ -338,17 +298,6 @@ public void testSavepointRescalingNonPartitionedStateCausesException() throws Ex } else { throw exception; } - } finally { - // clear any left overs from a possibly failed job - if (jobID != null && jobManager != null) { - Future jobRemovedFuture = jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), timeout); - - try { - Await.ready(jobRemovedFuture, timeout); - } catch (TimeoutException | InterruptedException ie) { - fail("Failed while cleaning up the cluster."); - } - } } } @@ -367,14 +316,12 @@ public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Excep int parallelism2 = numSlots; int maxParallelism = 13; - FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES); - Deadline deadline = timeout.fromNow(); + Duration timeout = Duration.ofMinutes(3); + Deadline deadline = Deadline.now().plus(timeout); - ActorGateway jobManager = null; - JobID jobID = null; + ClusterClient client = cluster.getClusterClient(); try { - jobManager = cluster.getLeaderGateway(deadline.timeLeft()); JobGraph jobGraph = createJobGraphWithKeyedAndNonPartitionedOperatorState( parallelism, @@ -385,9 +332,10 @@ public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Excep false, 100); - jobID = jobGraph.getJobID(); + final JobID jobID = jobGraph.getJobID(); - cluster.submitJobDetached(jobGraph); + client.setDetached(true); + client.submitJob(jobGraph, RescalingITCase.class.getClassLoader()); // wait til the sources have emitted numberElements for each key and completed a checkpoint SubtaskIndexFlatMapper.workCompletedLatch.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); @@ -409,22 +357,15 @@ public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Excep // clear the CollectionSink set for the restarted job CollectionSink.clearElementsSet(); - Future savepointPathFuture = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID, Option.empty()), deadline.timeLeft()); - - final String savepointPath = ((JobManagerMessages.TriggerSavepointSuccess) - Await.result(savepointPathFuture, deadline.timeLeft())).savepointPath(); - - Future jobRemovedFuture = jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), deadline.timeLeft()); - - Future cancellationResponseFuture = jobManager.ask(new JobManagerMessages.CancelJob(jobID), deadline.timeLeft()); + CompletableFuture savepointPathFuture = client.triggerSavepoint(jobID, null); - Object cancellationResponse = Await.result(cancellationResponseFuture, deadline.timeLeft()); + final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - assertTrue(cancellationResponse instanceof JobManagerMessages.CancellationSuccess); + client.cancel(jobID); - Await.ready(jobRemovedFuture, deadline.timeLeft()); - - jobID = null; + while (!getRunningJobs(client).isEmpty()) { + Thread.sleep(50); + } JobGraph scaledJobGraph = createJobGraphWithKeyedAndNonPartitionedOperatorState( parallelism2, @@ -437,11 +378,8 @@ public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Excep scaledJobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - jobID = scaledJobGraph.getJobID(); - - cluster.submitJobAndWait(scaledJobGraph, false); - - jobID = null; + client.setDetached(false); + client.submitJob(scaledJobGraph, RescalingITCase.class.getClassLoader()); Set> actualResult2 = CollectionSink.getElementsSet(); @@ -457,17 +395,6 @@ public void testSavepointRescalingWithKeyedAndNonPartitionedState() throws Excep } finally { // clear the CollectionSink set for the restarted job CollectionSink.clearElementsSet(); - - // clear any left overs from a possibly failed job - if (jobID != null && jobManager != null) { - Future jobRemovedFuture = jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), timeout); - - try { - Await.ready(jobRemovedFuture, timeout); - } catch (TimeoutException | InterruptedException ie) { - fail("Failed while cleaning up the cluster."); - } - } } } @@ -510,11 +437,10 @@ public void testSavepointRescalingPartitionedOperatorState(boolean scaleOut, Ope final int parallelism2 = scaleOut ? numSlots / 2 : numSlots; final int maxParallelism = 13; - FiniteDuration timeout = new FiniteDuration(3, TimeUnit.MINUTES); - Deadline deadline = timeout.fromNow(); + Duration timeout = Duration.ofMinutes(3); + Deadline deadline = Deadline.now().plus(timeout); - JobID jobID = null; - ActorGateway jobManager = null; + ClusterClient client = cluster.getClusterClient(); int counterSize = Math.max(parallelism, parallelism2); @@ -528,54 +454,44 @@ public void testSavepointRescalingPartitionedOperatorState(boolean scaleOut, Ope } try { - jobManager = cluster.getLeaderGateway(deadline.timeLeft()); - JobGraph jobGraph = createJobGraphWithOperatorState(parallelism, maxParallelism, checkpointMethod); - jobID = jobGraph.getJobID(); - - cluster.submitJobDetached(jobGraph); + final JobID jobID = jobGraph.getJobID(); - Object savepointResponse = null; + client.setDetached(true); + client.submitJob(jobGraph, RescalingITCase.class.getClassLoader()); // wait until the operator is started StateSourceBase.workStartedLatch.await(); - while (deadline.hasTimeLeft()) { - - Future savepointPathFuture = jobManager.ask(new JobManagerMessages.TriggerSavepoint(jobID, Option.empty()), deadline.timeLeft()); - FiniteDuration waitingTime = new FiniteDuration(10, TimeUnit.SECONDS); - savepointResponse = Await.result(savepointPathFuture, waitingTime); - - if (savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess) { - break; - } - } - - assertTrue(savepointResponse instanceof JobManagerMessages.TriggerSavepointSuccess); - - final String savepointPath = ((JobManagerMessages.TriggerSavepointSuccess) savepointResponse).savepointPath(); - - Future jobRemovedFuture = jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), deadline.timeLeft()); - - Future cancellationResponseFuture = jobManager.ask(new JobManagerMessages.CancelJob(jobID), deadline.timeLeft()); - - Object cancellationResponse = Await.result(cancellationResponseFuture, deadline.timeLeft()); + CompletableFuture savepointPathFuture = FutureUtils.retryWithDelay( + () -> { + try { + return client.triggerSavepoint(jobID, null); + } catch (FlinkException e) { + return FutureUtils.completedExceptionally(e); + } + }, + (int) deadline.timeLeft().getSeconds() / 10, + Time.seconds(10), + (throwable) -> true, + TestingUtils.defaultScheduledExecutor() + ); - assertTrue(cancellationResponse instanceof JobManagerMessages.CancellationSuccess); + final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - Await.ready(jobRemovedFuture, deadline.timeLeft()); + client.cancel(jobID); - // job successfully removed - jobID = null; + while (!getRunningJobs(client).isEmpty()) { + Thread.sleep(50); + } JobGraph scaledJobGraph = createJobGraphWithOperatorState(parallelism2, maxParallelism, checkpointMethod); scaledJobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(savepointPath)); - jobID = scaledJobGraph.getJobID(); - - cluster.submitJobAndWait(scaledJobGraph, false); + client.setDetached(false); + client.submitJob(scaledJobGraph, RescalingITCase.class.getClassLoader()); int sumExp = 0; int sumAct = 0; @@ -609,19 +525,7 @@ public void testSavepointRescalingPartitionedOperatorState(boolean scaleOut, Ope } assertEquals(sumExp, sumAct); - jobID = null; - } finally { - // clear any left overs from a possibly failed job - if (jobID != null && jobManager != null) { - Future jobRemovedFuture = jobManager.ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobID), timeout); - - try { - Await.ready(jobRemovedFuture, timeout); - } catch (TimeoutException | InterruptedException ie) { - fail("Failed while cleaning up the cluster."); - } - } } } @@ -1028,4 +932,12 @@ public void initializeState(FunctionInitializationContext context) throws Except } } } + + private static List getRunningJobs(ClusterClient client) throws Exception { + Collection statusMessages = client.listJobs().get(); + return statusMessages.stream() + .filter(status -> !status.getJobState().isGloballyTerminalState()) + .map(JobStatusMessage::getJobId) + .collect(Collectors.toList()); + } } From 765bcc154cc10133b2a9ceb884c62138560370e0 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 19 Mar 2018 13:59:22 +0100 Subject: [PATCH 201/268] [FLINK-8959][tests] Port AccumulatorLiveITCase to flip6 This closes #5719. --- .../accumulators/AccumulatorLiveITCase.java | 336 +++++---------- .../LegacyAccumulatorLiveITCase.java | 386 ++++++++++++++++++ 2 files changed, 482 insertions(+), 240 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/accumulators/LegacyAccumulatorLiveITCase.java diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index 756b81e095c09..ff362dde50246 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -18,292 +18,199 @@ package org.apache.flink.test.accumulators; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.Plan; -import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.IntCounter; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.LocalEnvironment; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.AkkaOptions; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HeartbeatManagerOptions; +import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.optimizer.DataStatistics; import org.apache.flink.optimizer.Optimizer; import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.akka.ListeningBehaviour; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.instance.ActorGateway; -import org.apache.flink.runtime.instance.AkkaActorGateway; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.messages.JobManagerMessages; -import org.apache.flink.runtime.testingUtils.TestingCluster; -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; -import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.MiniClusterResource; +import org.apache.flink.testutils.category.Flip6; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.pattern.Patterns; -import akka.testkit.JavaTestKit; -import akka.util.Timeout; -import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.time.Duration; import java.util.ArrayList; import java.util.List; -import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.FiniteDuration; - -import static org.junit.Assert.fail; - /** - * Tests the availability of accumulator results during runtime. The test case tests a user-defined - * accumulator and Flink's internal accumulators for two consecutive tasks. - * - *

    CHAINED[Source -> Map] -> Sink - * - *

    Checks are performed as the elements arrive at the operators. Checks consist of a message sent by - * the task to the task manager which notifies the job manager and sends the current accumulators. - * The task blocks until the test has been notified about the current accumulator values. - * - *

    A barrier between the operators ensures that that pipelining is disabled for the streaming test. - * The batch job reads the records one at a time. The streaming code buffers the records beforehand; - * that's why exact guarantees about the number of records read are very hard to make. Thus, why we - * check for an upper bound of the elements read. + * Tests the availability of accumulator results during runtime. */ +@Category(Flip6.class) public class AccumulatorLiveITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(AccumulatorLiveITCase.class); - private static ActorSystem system; - private static ActorGateway jobManagerGateway; - private static ActorRef taskManager; - - private static JobID jobID; - private static JobGraph jobGraph; - // name of user accumulator private static final String ACCUMULATOR_NAME = "test"; + private static final long HEARTBEAT_INTERVAL = 50L; + // number of heartbeat intervals to check private static final int NUM_ITERATIONS = 5; - private static List inputData = new ArrayList<>(NUM_ITERATIONS); + private static final List inputData = new ArrayList<>(NUM_ITERATIONS); - private static final FiniteDuration TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS); + static { + // generate test data + for (int i = 0; i < NUM_ITERATIONS; i++) { + inputData.add(i); + } + } - @Before - public void before() throws Exception { - system = AkkaUtils.createLocalActorSystem(new Configuration()); + @ClassRule + public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfiguration(), + 1, + 1), + true); + private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); config.setString(AkkaOptions.ASK_TIMEOUT, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT()); - TestingCluster testingCluster = new TestingCluster(config, false, true); - testingCluster.start(); - - jobManagerGateway = testingCluster.getLeaderGateway(TestingUtils.TESTING_DURATION()); - taskManager = testingCluster.getTaskManagersAsJava().get(0); - - // generate test data - for (int i = 0; i < NUM_ITERATIONS; i++) { - inputData.add(i, String.valueOf(i + 1)); - } + config.setLong(HeartbeatManagerOptions.HEARTBEAT_INTERVAL, HEARTBEAT_INTERVAL); - NotifyingMapper.finished = false; + return config; } - @After - public void after() throws Exception { - JavaTestKit.shutdownActorSystem(system); - inputData.clear(); + @Before + public void resetLatches() throws InterruptedException { + NotifyingMapper.reset(); } @Test public void testBatch() throws Exception { - - /** The program **/ - ExecutionEnvironment env = new BatchPlanExtractor(); + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); - DataSet input = env.fromCollection(inputData); + DataSet input = env.fromCollection(inputData); input .flatMap(new NotifyingMapper()) - .output(new NotifyingOutputFormat()); - - env.execute(); + .output(new DummyOutputFormat()); // Extract job graph and set job id for the task to notify of accumulator changes. - jobGraph = getOptimizedPlan(((BatchPlanExtractor) env).plan); - jobID = jobGraph.getJobID(); + JobGraph jobGraph = getJobGraph(env.createProgramPlan()); - verifyResults(); + submitJobAndVerifyResults(jobGraph); } @Test public void testStreaming() throws Exception { - StreamExecutionEnvironment env = new DummyStreamExecutionEnvironment(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); - DataStream input = env.fromCollection(inputData); + DataStream input = env.fromCollection(inputData); input .flatMap(new NotifyingMapper()) - .writeUsingOutputFormat(new NotifyingOutputFormat()).disableChaining(); + .writeUsingOutputFormat(new DummyOutputFormat()).disableChaining(); - jobGraph = env.getStreamGraph().getJobGraph(); - jobID = jobGraph.getJobID(); + JobGraph jobGraph = env.getStreamGraph().getJobGraph(); - verifyResults(); + submitJobAndVerifyResults(jobGraph); } - private static void verifyResults() { - new JavaTestKit(system) {{ - - ActorGateway selfGateway = new AkkaActorGateway(getRef(), jobManagerGateway.leaderSessionID()); - - // register for accumulator changes - jobManagerGateway.tell(new TestingJobManagerMessages.NotifyWhenAccumulatorChange(jobID), selfGateway); - expectMsgEquals(TIMEOUT, true); - - // submit job - - jobManagerGateway.tell( - new JobManagerMessages.SubmitJob( - jobGraph, - ListeningBehaviour.EXECUTION_RESULT), - selfGateway); - expectMsgClass(TIMEOUT, JobManagerMessages.JobSubmitSuccess.class); - - TestingJobManagerMessages.UpdatedAccumulators msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); - Map> userAccumulators = msg.userAccumulators(); - - ExecutionAttemptID mapperTaskID = null; - - ExecutionAttemptID sinkTaskID = null; - - /* Check for accumulator values */ - if (checkUserAccumulators(0, userAccumulators)) { - LOG.info("Passed initial check for map task."); - } else { - fail("Wrong accumulator results when map task begins execution."); - } - - int expectedAccVal = 0; - - /* for mapper task */ - for (int i = 1; i <= NUM_ITERATIONS; i++) { - expectedAccVal += i; - - // receive message - msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); - userAccumulators = msg.userAccumulators(); - - LOG.info("{}", userAccumulators); - - if (checkUserAccumulators(expectedAccVal, userAccumulators)) { - LOG.info("Passed round #" + i); - } else if (checkUserAccumulators(expectedAccVal, userAccumulators)) { - // we determined the wrong task id and need to switch the two here - ExecutionAttemptID temp = mapperTaskID; - mapperTaskID = sinkTaskID; - sinkTaskID = temp; - LOG.info("Passed round #" + i); - } else { - fail("Failed in round #" + i); - } - } - - msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); - userAccumulators = msg.userAccumulators(); - - if (checkUserAccumulators(expectedAccVal, userAccumulators)) { - LOG.info("Passed initial check for sink task."); - } else { - fail("Wrong accumulator results when sink task begins execution."); - } - - /* for sink task */ - for (int i = 1; i <= NUM_ITERATIONS; i++) { - - // receive message - msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); - - userAccumulators = msg.userAccumulators(); - - LOG.info("{}", userAccumulators); - - if (checkUserAccumulators(expectedAccVal, userAccumulators)) { - LOG.info("Passed round #" + i); - } else { - fail("Failed in round #" + i); - } - } - - expectMsgClass(TIMEOUT, JobManagerMessages.JobResultSuccess.class); - - }}; - } - - private static boolean checkUserAccumulators(int expected, Map> accumulatorMap) { - LOG.info("checking user accumulators"); - return accumulatorMap.containsKey(ACCUMULATOR_NAME) && expected == ((IntCounter) accumulatorMap.get(ACCUMULATOR_NAME)).getLocalValue(); + private static void submitJobAndVerifyResults(JobGraph jobGraph) throws Exception { + Deadline deadline = Deadline.now().plus(Duration.ofSeconds(30)); + + ClusterClient client = MINI_CLUSTER_RESOURCE.getClusterClient(); + + client.setDetached(true); + client.submitJob(jobGraph, AccumulatorLiveITCase.class.getClassLoader()); + + try { + NotifyingMapper.notifyLatch.await(); + + FutureUtils.retrySuccesfulWithDelay( + () -> { + try { + return CompletableFuture.completedFuture(client.getAccumulators(jobGraph.getJobID())); + } catch (Exception e) { + return FutureUtils.completedExceptionally(e); + } + }, + Time.milliseconds(20), + deadline, + accumulators -> accumulators.size() == 1 + && accumulators.containsKey(ACCUMULATOR_NAME) + && (int) accumulators.get(ACCUMULATOR_NAME) == NUM_ITERATIONS, + TestingUtils.defaultScheduledExecutor() + ).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + + NotifyingMapper.shutdownLatch.trigger(); + } finally { + NotifyingMapper.shutdownLatch.trigger(); + } } /** * UDF that notifies when it changes the accumulator values. */ - private static class NotifyingMapper extends RichFlatMapFunction { + private static class NotifyingMapper extends RichFlatMapFunction { private static final long serialVersionUID = 1L; - private IntCounter counter = new IntCounter(); + private static final OneShotLatch notifyLatch = new OneShotLatch(); + private static final OneShotLatch shutdownLatch = new OneShotLatch(); - private static boolean finished = false; + private final IntCounter counter = new IntCounter(); @Override public void open(Configuration parameters) throws Exception { getRuntimeContext().addAccumulator(ACCUMULATOR_NAME, counter); - notifyTaskManagerOfAccumulatorUpdate(); } @Override - public void flatMap(String value, Collector out) throws Exception { - int val = Integer.valueOf(value); - counter.add(val); - out.collect(val); + public void flatMap(Integer value, Collector out) throws Exception { + counter.add(1); + out.collect(value); LOG.debug("Emitting value {}.", value); - notifyTaskManagerOfAccumulatorUpdate(); + if (counter.getLocalValuePrimitive() == 5) { + notifyLatch.trigger(); + } } @Override public void close() throws Exception { - finished = true; + shutdownLatch.await(); + } + + private static void reset() throws InterruptedException { + notifyLatch.reset(); + shutdownLatch.reset(); } } /** - * Outputs format which notifies of accumulator changes and waits for the previous mapper. + * Outputs format which waits for the previous mapper. */ - private static class NotifyingOutputFormat implements OutputFormat { + private static class DummyOutputFormat implements OutputFormat { private static final long serialVersionUID = 1L; @Override @@ -312,17 +219,10 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { - while (!NotifyingMapper.finished) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) {} - } - notifyTaskManagerOfAccumulatorUpdate(); } @Override public void writeRecord(Integer record) throws IOException { - notifyTaskManagerOfAccumulatorUpdate(); } @Override @@ -330,57 +230,13 @@ public void close() throws IOException { } } - /** - * Notify task manager of accumulator update and wait until the Heartbeat containing the message - * has been reported. - */ - public static void notifyTaskManagerOfAccumulatorUpdate() { - new JavaTestKit(system) {{ - Timeout timeout = new Timeout(TIMEOUT); - Future ask = Patterns.ask(taskManager, new TestingTaskManagerMessages.AccumulatorsChanged(jobID), timeout); - try { - Await.result(ask, timeout.duration()); - } catch (Exception e) { - fail("Failed to notify task manager of accumulator update."); - } - }}; - } - /** * Helpers to generate the JobGraph. */ - private static JobGraph getOptimizedPlan(Plan plan) { + private static JobGraph getJobGraph(Plan plan) { Optimizer pc = new Optimizer(new DataStatistics(), new Configuration()); JobGraphGenerator jgg = new JobGraphGenerator(); OptimizedPlan op = pc.compile(plan); return jgg.compileJobGraph(op); } - - private static class BatchPlanExtractor extends LocalEnvironment { - - private Plan plan = null; - - @Override - public JobExecutionResult execute(String jobName) throws Exception { - plan = createProgramPlan(); - return new JobExecutionResult(new JobID(), -1, null); - } - } - - /** - * This is used to for creating the example topology. {@link #execute} is never called, we - * only use this to call {@link #getStreamGraph()}. - */ - private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute() throws Exception { - return execute("default"); - } - - @Override - public JobExecutionResult execute(String jobName) throws Exception { - throw new RuntimeException("This should not be called."); - } - } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/LegacyAccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/LegacyAccumulatorLiveITCase.java new file mode 100644 index 0000000000000..6595b100c1581 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/LegacyAccumulatorLiveITCase.java @@ -0,0 +1,386 @@ +/* + * 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.test.accumulators; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.Plan; +import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.LocalEnvironment; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.optimizer.DataStatistics; +import org.apache.flink.optimizer.Optimizer; +import org.apache.flink.optimizer.plan.OptimizedPlan; +import org.apache.flink.optimizer.plantranslate.JobGraphGenerator; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.akka.ListeningBehaviour; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.instance.AkkaActorGateway; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.testingUtils.TestingCluster; +import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; +import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.Collector; +import org.apache.flink.util.TestLogger; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.pattern.Patterns; +import akka.testkit.JavaTestKit; +import akka.util.Timeout; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import static org.junit.Assert.fail; + +/** + * Tests the availability of accumulator results during runtime. The test case tests a user-defined + * accumulator and Flink's internal accumulators for two consecutive tasks. + * + *

    CHAINED[Source -> Map] -> Sink + * + *

    Checks are performed as the elements arrive at the operators. Checks consist of a message sent by + * the task to the task manager which notifies the job manager and sends the current accumulators. + * The task blocks until the test has been notified about the current accumulator values. + * + *

    A barrier between the operators ensures that that pipelining is disabled for the streaming test. + * The batch job reads the records one at a time. The streaming code buffers the records beforehand; + * that's why exact guarantees about the number of records read are very hard to make. Thus, why we + * check for an upper bound of the elements read. + */ +public class LegacyAccumulatorLiveITCase extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(LegacyAccumulatorLiveITCase.class); + + private static ActorSystem system; + private static ActorGateway jobManagerGateway; + private static ActorRef taskManager; + + private static JobID jobID; + private static JobGraph jobGraph; + + // name of user accumulator + private static final String ACCUMULATOR_NAME = "test"; + + // number of heartbeat intervals to check + private static final int NUM_ITERATIONS = 5; + + private static List inputData = new ArrayList<>(NUM_ITERATIONS); + + private static final FiniteDuration TIMEOUT = new FiniteDuration(10, TimeUnit.SECONDS); + + @Before + public void before() throws Exception { + system = AkkaUtils.createLocalActorSystem(new Configuration()); + + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + config.setString(AkkaOptions.ASK_TIMEOUT, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT()); + TestingCluster testingCluster = new TestingCluster(config, false, true); + testingCluster.start(); + + jobManagerGateway = testingCluster.getLeaderGateway(TestingUtils.TESTING_DURATION()); + taskManager = testingCluster.getTaskManagersAsJava().get(0); + + // generate test data + for (int i = 0; i < NUM_ITERATIONS; i++) { + inputData.add(i, String.valueOf(i + 1)); + } + + NotifyingMapper.finished = false; + } + + @After + public void after() throws Exception { + JavaTestKit.shutdownActorSystem(system); + inputData.clear(); + } + + @Test + public void testBatch() throws Exception { + + /** The program **/ + ExecutionEnvironment env = new BatchPlanExtractor(); + env.setParallelism(1); + + DataSet input = env.fromCollection(inputData); + input + .flatMap(new NotifyingMapper()) + .output(new NotifyingOutputFormat()); + + env.execute(); + + // Extract job graph and set job id for the task to notify of accumulator changes. + jobGraph = getOptimizedPlan(((BatchPlanExtractor) env).plan); + jobID = jobGraph.getJobID(); + + verifyResults(); + } + + @Test + public void testStreaming() throws Exception { + + StreamExecutionEnvironment env = new DummyStreamExecutionEnvironment(); + env.setParallelism(1); + + DataStream input = env.fromCollection(inputData); + input + .flatMap(new NotifyingMapper()) + .writeUsingOutputFormat(new NotifyingOutputFormat()).disableChaining(); + + jobGraph = env.getStreamGraph().getJobGraph(); + jobID = jobGraph.getJobID(); + + verifyResults(); + } + + private static void verifyResults() { + new JavaTestKit(system) {{ + + ActorGateway selfGateway = new AkkaActorGateway(getRef(), jobManagerGateway.leaderSessionID()); + + // register for accumulator changes + jobManagerGateway.tell(new TestingJobManagerMessages.NotifyWhenAccumulatorChange(jobID), selfGateway); + expectMsgEquals(TIMEOUT, true); + + // submit job + + jobManagerGateway.tell( + new JobManagerMessages.SubmitJob( + jobGraph, + ListeningBehaviour.EXECUTION_RESULT), + selfGateway); + expectMsgClass(TIMEOUT, JobManagerMessages.JobSubmitSuccess.class); + + TestingJobManagerMessages.UpdatedAccumulators msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); + Map> userAccumulators = msg.userAccumulators(); + + ExecutionAttemptID mapperTaskID = null; + + ExecutionAttemptID sinkTaskID = null; + + /* Check for accumulator values */ + if (checkUserAccumulators(0, userAccumulators)) { + LOG.info("Passed initial check for map task."); + } else { + fail("Wrong accumulator results when map task begins execution."); + } + + int expectedAccVal = 0; + + /* for mapper task */ + for (int i = 1; i <= NUM_ITERATIONS; i++) { + expectedAccVal += i; + + // receive message + msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); + userAccumulators = msg.userAccumulators(); + + LOG.info("{}", userAccumulators); + + if (checkUserAccumulators(expectedAccVal, userAccumulators)) { + LOG.info("Passed round #" + i); + } else if (checkUserAccumulators(expectedAccVal, userAccumulators)) { + // we determined the wrong task id and need to switch the two here + ExecutionAttemptID temp = mapperTaskID; + mapperTaskID = sinkTaskID; + sinkTaskID = temp; + LOG.info("Passed round #" + i); + } else { + fail("Failed in round #" + i); + } + } + + msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); + userAccumulators = msg.userAccumulators(); + + if (checkUserAccumulators(expectedAccVal, userAccumulators)) { + LOG.info("Passed initial check for sink task."); + } else { + fail("Wrong accumulator results when sink task begins execution."); + } + + /* for sink task */ + for (int i = 1; i <= NUM_ITERATIONS; i++) { + + // receive message + msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); + + userAccumulators = msg.userAccumulators(); + + LOG.info("{}", userAccumulators); + + if (checkUserAccumulators(expectedAccVal, userAccumulators)) { + LOG.info("Passed round #" + i); + } else { + fail("Failed in round #" + i); + } + } + + expectMsgClass(TIMEOUT, JobManagerMessages.JobResultSuccess.class); + + }}; + } + + private static boolean checkUserAccumulators(int expected, Map> accumulatorMap) { + LOG.info("checking user accumulators"); + return accumulatorMap.containsKey(ACCUMULATOR_NAME) && expected == ((IntCounter) accumulatorMap.get(ACCUMULATOR_NAME)).getLocalValue(); + } + + /** + * UDF that notifies when it changes the accumulator values. + */ + private static class NotifyingMapper extends RichFlatMapFunction { + private static final long serialVersionUID = 1L; + + private IntCounter counter = new IntCounter(); + + private static boolean finished = false; + + @Override + public void open(Configuration parameters) throws Exception { + getRuntimeContext().addAccumulator(ACCUMULATOR_NAME, counter); + notifyTaskManagerOfAccumulatorUpdate(); + } + + @Override + public void flatMap(String value, Collector out) throws Exception { + int val = Integer.valueOf(value); + counter.add(val); + out.collect(val); + LOG.debug("Emitting value {}.", value); + notifyTaskManagerOfAccumulatorUpdate(); + } + + @Override + public void close() throws Exception { + finished = true; + } + } + + /** + * Outputs format which notifies of accumulator changes and waits for the previous mapper. + */ + private static class NotifyingOutputFormat implements OutputFormat { + private static final long serialVersionUID = 1L; + + @Override + public void configure(Configuration parameters) { + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + while (!NotifyingMapper.finished) { + try { + Thread.sleep(1000); + } catch (InterruptedException e) {} + } + notifyTaskManagerOfAccumulatorUpdate(); + } + + @Override + public void writeRecord(Integer record) throws IOException { + notifyTaskManagerOfAccumulatorUpdate(); + } + + @Override + public void close() throws IOException { + } + } + + /** + * Notify task manager of accumulator update and wait until the Heartbeat containing the message + * has been reported. + */ + public static void notifyTaskManagerOfAccumulatorUpdate() { + new JavaTestKit(system) {{ + Timeout timeout = new Timeout(TIMEOUT); + Future ask = Patterns.ask(taskManager, new TestingTaskManagerMessages.AccumulatorsChanged(jobID), timeout); + try { + Await.result(ask, timeout.duration()); + } catch (Exception e) { + fail("Failed to notify task manager of accumulator update."); + } + }}; + } + + /** + * Helpers to generate the JobGraph. + */ + private static JobGraph getOptimizedPlan(Plan plan) { + Optimizer pc = new Optimizer(new DataStatistics(), new Configuration()); + JobGraphGenerator jgg = new JobGraphGenerator(); + OptimizedPlan op = pc.compile(plan); + return jgg.compileJobGraph(op); + } + + private static class BatchPlanExtractor extends LocalEnvironment { + + private Plan plan = null; + + @Override + public JobExecutionResult execute(String jobName) throws Exception { + plan = createProgramPlan(); + return new JobExecutionResult(new JobID(), -1, null); + } + } + + /** + * This is used to for creating the example topology. {@link #execute} is never called, we + * only use this to call {@link #getStreamGraph()}. + */ + private static class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment { + + @Override + public JobExecutionResult execute() throws Exception { + return execute("default"); + } + + @Override + public JobExecutionResult execute(String jobName) throws Exception { + throw new RuntimeException("This should not be called."); + } + } +} From ecdeb35cf43b160b3ff86d8acfb9672e758bd2b7 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 19 Mar 2018 14:17:34 +0100 Subject: [PATCH 202/268] [FLINK-8957][tests] Port JMXJobManagerMetricTest to flip6 This closes #5720. --- flink-metrics/flink-metrics-jmx/pom.xml | 6 ++ .../jobmanager/JMXJobManagerMetricTest.java | 69 ++++++++++--------- 2 files changed, 44 insertions(+), 31 deletions(-) diff --git a/flink-metrics/flink-metrics-jmx/pom.xml b/flink-metrics/flink-metrics-jmx/pom.xml index 66de1b3f90b8f..d90595c0cb238 100644 --- a/flink-metrics/flink-metrics-jmx/pom.xml +++ b/flink-metrics/flink-metrics-jmx/pom.xml @@ -85,5 +85,11 @@ under the License. ${project.version} test + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java index 6770ec326de64..c00b5d357cb29 100644 --- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java +++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java @@ -18,38 +18,40 @@ package org.apache.flink.runtime.jobmanager; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MetricOptions; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.metrics.jmx.JMXReporter; import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; -import org.apache.flink.runtime.testingUtils.TestingCluster; -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.test.util.MiniClusterResource; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import javax.management.MBeanServer; import javax.management.ObjectName; import java.lang.management.ManagementFactory; +import java.time.Duration; import java.util.Collections; import java.util.Set; import java.util.concurrent.TimeUnit; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; - import static org.junit.Assert.assertEquals; /** @@ -57,24 +59,31 @@ */ public class JMXJobManagerMetricTest { - /** - * Tests that metrics registered on the JobManager are actually accessible via JMX. - */ - @Test - public void testJobManagerJMXMetricAccess() throws Exception { - Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); + @ClassRule + public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfiguration(), + 1, + 1), + true); + + private static Configuration getConfiguration() { Configuration flinkConfiguration = new Configuration(); flinkConfiguration.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName()); - flinkConfiguration.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test.port", "9060-9075"); - flinkConfiguration.setString(MetricOptions.SCOPE_NAMING_JM_JOB, "jobmanager."); - TestingCluster flink = new TestingCluster(flinkConfiguration); + return flinkConfiguration; + } - try { - flink.start(); + /** + * Tests that metrics registered on the JobManager are actually accessible via JMX. + */ + @Test + public void testJobManagerJMXMetricAccess() throws Exception { + Deadline deadline = Deadline.now().plus(Duration.ofMinutes(2)); + try { JobVertex sourceJobVertex = new JobVertex("Source"); sourceJobVertex.setInvokableClass(BlockingInvokable.class); @@ -92,28 +101,26 @@ public void testJobManagerJMXMetricAccess() throws Exception { true), null)); - flink.waitForActorsToBeAlive(); - - flink.submitJobDetached(jobGraph); + ClusterClient client = MINI_CLUSTER_RESOURCE.getClusterClient(); + client.setDetached(true); + client.submitJob(jobGraph, JMXJobManagerMetricTest.class.getClassLoader()); - Future jobRunning = flink.getLeaderGateway(deadline.timeLeft()) - .ask(new TestingJobManagerMessages.WaitForAllVerticesToBeRunning(jobGraph.getJobID()), deadline.timeLeft()); - Await.ready(jobRunning, deadline.timeLeft()); + FutureUtils.retrySuccesfulWithDelay( + () -> client.getJobStatus(jobGraph.getJobID()), + Time.milliseconds(10), + deadline, + status -> status == JobStatus.RUNNING, + TestingUtils.defaultScheduledExecutor() + ).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer(); Set nameSet = mBeanServer.queryNames(new ObjectName("org.apache.flink.jobmanager.job.lastCheckpointSize:job_name=TestingJob,*"), null); Assert.assertEquals(1, nameSet.size()); assertEquals(-1L, mBeanServer.getAttribute(nameSet.iterator().next(), "Value")); - Future jobFinished = flink.getLeaderGateway(deadline.timeLeft()) - .ask(new TestingJobManagerMessages.NotifyWhenJobRemoved(jobGraph.getJobID()), deadline.timeLeft()); - BlockingInvokable.unblock(); - - // wait til the job has finished - Await.ready(jobFinished, deadline.timeLeft()); } finally { - flink.stop(); + BlockingInvokable.unblock(); } } From 4ebbbe7dc4bbf1da66449acba0b5af2feb409de6 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 19 Mar 2018 15:16:18 +0100 Subject: [PATCH 203/268] [FLINK-8958][tests] Port TaskCancelAsyncProducerConsumerITCase to flip6 This closes #5722. --- ...TaskCancelAsyncProducerConsumerITCase.java | 287 ++++++++++++++++++ ...TaskCancelAsyncProducerConsumerITCase.java | 82 ++--- 2 files changed, 329 insertions(+), 40 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LegacyTaskCancelAsyncProducerConsumerITCase.java diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LegacyTaskCancelAsyncProducerConsumerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LegacyTaskCancelAsyncProducerConsumerITCase.java new file mode 100644 index 0000000000000..ee0bfda39671d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/LegacyTaskCancelAsyncProducerConsumerITCase.java @@ -0,0 +1,287 @@ +/* + * 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.taskmanager; + +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.instance.ActorGateway; +import org.apache.flink.runtime.io.network.api.writer.RecordWriter; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.jobgraph.DistributionPattern; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; +import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; +import org.apache.flink.runtime.testingUtils.TestingCluster; +import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobStatus; +import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning; +import org.apache.flink.types.LongValue; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.concurrent.TimeUnit; + +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; + +import static org.apache.flink.runtime.io.network.buffer.LocalBufferPoolDestroyTest.isInBlockingBufferRequest; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class LegacyTaskCancelAsyncProducerConsumerITCase extends TestLogger { + + // The Exceptions thrown by the producer/consumer Threads + private static volatile Exception ASYNC_PRODUCER_EXCEPTION; + private static volatile Exception ASYNC_CONSUMER_EXCEPTION; + + // The Threads producing/consuming the intermediate stream + private static volatile Thread ASYNC_PRODUCER_THREAD; + private static volatile Thread ASYNC_CONSUMER_THREAD; + + /** + * Tests that a task waiting on an async producer/consumer that is stuck + * in a blocking buffer request can be properly cancelled. + * + *

    This is currently required for the Flink Kafka sources, which spawn + * a separate Thread consuming from Kafka and producing the intermediate + * streams in the spawned Thread instead of the main task Thread. + */ + @Test + public void testCancelAsyncProducerAndConsumer() throws Exception { + Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); + TestingCluster flink = null; + + try { + // Cluster + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); + config.setInteger(TaskManagerOptions.MEMORY_SEGMENT_SIZE, 4096); + config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 9); + + flink = new TestingCluster(config, true); + flink.start(); + + // Job with async producer and consumer + JobVertex producer = new JobVertex("AsyncProducer"); + producer.setParallelism(1); + producer.setInvokableClass(AsyncProducer.class); + + JobVertex consumer = new JobVertex("AsyncConsumer"); + consumer.setParallelism(1); + consumer.setInvokableClass(AsyncConsumer.class); + consumer.connectNewDataSetAsInput(producer, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); + + SlotSharingGroup slot = new SlotSharingGroup(producer.getID(), consumer.getID()); + producer.setSlotSharingGroup(slot); + consumer.setSlotSharingGroup(slot); + + JobGraph jobGraph = new JobGraph(producer, consumer); + + // Submit job and wait until running + ActorGateway jobManager = flink.getLeaderGateway(deadline.timeLeft()); + flink.submitJobDetached(jobGraph); + + Object msg = new WaitForAllVerticesToBeRunning(jobGraph.getJobID()); + Future runningFuture = jobManager.ask(msg, deadline.timeLeft()); + Await.ready(runningFuture, deadline.timeLeft()); + + // Wait for blocking requests, cancel and wait for cancellation + msg = new NotifyWhenJobStatus(jobGraph.getJobID(), JobStatus.CANCELED); + Future cancelledFuture = jobManager.ask(msg, deadline.timeLeft()); + + boolean producerBlocked = false; + for (int i = 0; i < 50; i++) { + Thread thread = ASYNC_PRODUCER_THREAD; + + if (thread != null && thread.isAlive()) { + StackTraceElement[] stackTrace = thread.getStackTrace(); + producerBlocked = isInBlockingBufferRequest(stackTrace); + } + + if (producerBlocked) { + break; + } else { + // Retry + Thread.sleep(500L); + } + } + + // Verify that async producer is in blocking request + assertTrue("Producer thread is not blocked: " + Arrays.toString(ASYNC_PRODUCER_THREAD.getStackTrace()), producerBlocked); + + boolean consumerWaiting = false; + for (int i = 0; i < 50; i++) { + Thread thread = ASYNC_CONSUMER_THREAD; + + if (thread != null && thread.isAlive()) { + consumerWaiting = thread.getState() == Thread.State.WAITING; + } + + if (consumerWaiting) { + break; + } else { + // Retry + Thread.sleep(500L); + } + } + + // Verify that async consumer is in blocking request + assertTrue("Consumer thread is not blocked.", consumerWaiting); + + msg = new CancelJob(jobGraph.getJobID()); + Future cancelFuture = jobManager.ask(msg, deadline.timeLeft()); + Await.ready(cancelFuture, deadline.timeLeft()); + + Await.ready(cancelledFuture, deadline.timeLeft()); + + // Verify the expected Exceptions + assertNotNull(ASYNC_PRODUCER_EXCEPTION); + assertEquals(IllegalStateException.class, ASYNC_PRODUCER_EXCEPTION.getClass()); + + assertNotNull(ASYNC_CONSUMER_EXCEPTION); + assertEquals(IllegalStateException.class, ASYNC_CONSUMER_EXCEPTION.getClass()); + } finally { + if (flink != null) { + flink.stop(); + } + } + } + + /** + * Invokable emitting records in a separate Thread (not the main Task + * thread). + */ + public static class AsyncProducer extends AbstractInvokable { + + public AsyncProducer(Environment environment) { + super(environment); + } + + @Override + public void invoke() throws Exception { + Thread producer = new ProducerThread(getEnvironment().getWriter(0)); + + // Publish the async producer for the main test Thread + ASYNC_PRODUCER_THREAD = producer; + + producer.start(); + + // Wait for the producer Thread to finish. This is executed in the + // main Task thread and will be interrupted on cancellation. + while (producer.isAlive()) { + try { + producer.join(); + } catch (InterruptedException ignored) { + } + } + } + + /** + * The Thread emitting the records. + */ + private static class ProducerThread extends Thread { + + private final RecordWriter recordWriter; + + public ProducerThread(ResultPartitionWriter partitionWriter) { + this.recordWriter = new RecordWriter<>(partitionWriter); + } + + @Override + public void run() { + LongValue current = new LongValue(0); + + try { + while (true) { + current.setValue(current.getValue() + 1); + recordWriter.emit(current); + recordWriter.flushAll(); + } + } catch (Exception e) { + ASYNC_PRODUCER_EXCEPTION = e; + } + } + } + } + + /** + * Invokable consuming buffers in a separate Thread (not the main Task + * thread). + */ + public static class AsyncConsumer extends AbstractInvokable { + + public AsyncConsumer(Environment environment) { + super(environment); + } + + @Override + public void invoke() throws Exception { + Thread consumer = new ConsumerThread(getEnvironment().getInputGate(0)); + + // Publish the async consumer for the main test Thread + ASYNC_CONSUMER_THREAD = consumer; + + consumer.start(); + + // Wait for the consumer Thread to finish. This is executed in the + // main Task thread and will be interrupted on cancellation. + while (consumer.isAlive()) { + try { + consumer.join(); + } catch (InterruptedException ignored) { + } + } + } + + /** + * The Thread consuming buffers. + */ + private static class ConsumerThread extends Thread { + + private final InputGate inputGate; + + public ConsumerThread(InputGate inputGate) { + this.inputGate = inputGate; + } + + @Override + public void run() { + try { + while (true) { + inputGate.getNextBufferOrEvent(); + } + } catch (Exception e) { + ASYNC_CONSUMER_EXCEPTION = e; + } + } + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java index c63af83dc20a5..4b73b0925ff50 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java @@ -18,11 +18,12 @@ package org.apache.flink.runtime.taskmanager; -import org.apache.flink.configuration.ConfigConstants; +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.TaskManagerOptions; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; @@ -33,28 +34,26 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; -import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob; -import org.apache.flink.runtime.testingUtils.TestingCluster; -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobStatus; -import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.testutils.category.Flip6; import org.apache.flink.types.LongValue; import org.apache.flink.util.TestLogger; import org.junit.Test; +import org.junit.experimental.categories.Category; +import java.time.Duration; import java.util.Arrays; import java.util.concurrent.TimeUnit; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; - import static org.apache.flink.runtime.io.network.buffer.LocalBufferPoolDestroyTest.isInBlockingBufferRequest; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +@Category(Flip6.class) public class TaskCancelAsyncProducerConsumerITCase extends TestLogger { // The Exceptions thrown by the producer/consumer Threads @@ -75,18 +74,20 @@ public class TaskCancelAsyncProducerConsumerITCase extends TestLogger { */ @Test public void testCancelAsyncProducerAndConsumer() throws Exception { - Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); - TestingCluster flink = null; - - try { - // Cluster - Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1); - config.setInteger(TaskManagerOptions.MEMORY_SEGMENT_SIZE, 4096); - config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 9); - - flink = new TestingCluster(config, true); + Deadline deadline = Deadline.now().plus(Duration.ofMinutes(2)); + + // Cluster + Configuration config = new Configuration(); + config.setInteger(TaskManagerOptions.MEMORY_SEGMENT_SIZE, 4096); + config.setInteger(TaskManagerOptions.NETWORK_NUM_BUFFERS, 9); + + MiniClusterConfiguration miniClusterConfiguration = new MiniClusterConfiguration.Builder() + .setConfiguration(config) + .setNumTaskManagers(1) + .setNumSlotsPerTaskManager(1) + .build(); + + try (MiniCluster flink = new MiniCluster(miniClusterConfiguration)) { flink.start(); // Job with async producer and consumer @@ -106,16 +107,15 @@ public void testCancelAsyncProducerAndConsumer() throws Exception { JobGraph jobGraph = new JobGraph(producer, consumer); // Submit job and wait until running - ActorGateway jobManager = flink.getLeaderGateway(deadline.timeLeft()); - flink.submitJobDetached(jobGraph); - - Object msg = new WaitForAllVerticesToBeRunning(jobGraph.getJobID()); - Future runningFuture = jobManager.ask(msg, deadline.timeLeft()); - Await.ready(runningFuture, deadline.timeLeft()); + flink.runDetached(jobGraph); - // Wait for blocking requests, cancel and wait for cancellation - msg = new NotifyWhenJobStatus(jobGraph.getJobID(), JobStatus.CANCELED); - Future cancelledFuture = jobManager.ask(msg, deadline.timeLeft()); + FutureUtils.retrySuccesfulWithDelay( + () -> flink.getJobStatus(jobGraph.getJobID()), + Time.milliseconds(10), + deadline, + status -> status == JobStatus.RUNNING, + TestingUtils.defaultScheduledExecutor() + ).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); boolean producerBlocked = false; for (int i = 0; i < 50; i++) { @@ -156,11 +156,17 @@ public void testCancelAsyncProducerAndConsumer() throws Exception { // Verify that async consumer is in blocking request assertTrue("Consumer thread is not blocked.", consumerWaiting); - msg = new CancelJob(jobGraph.getJobID()); - Future cancelFuture = jobManager.ask(msg, deadline.timeLeft()); - Await.ready(cancelFuture, deadline.timeLeft()); + flink.cancelJob(jobGraph.getJobID()) + .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); - Await.ready(cancelledFuture, deadline.timeLeft()); + // wait until the job is canceled + FutureUtils.retrySuccesfulWithDelay( + () -> flink.getJobStatus(jobGraph.getJobID()), + Time.milliseconds(10), + deadline, + status -> status == JobStatus.CANCELED, + TestingUtils.defaultScheduledExecutor() + ).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); // Verify the expected Exceptions assertNotNull(ASYNC_PRODUCER_EXCEPTION); @@ -168,10 +174,6 @@ public void testCancelAsyncProducerAndConsumer() throws Exception { assertNotNull(ASYNC_CONSUMER_EXCEPTION); assertEquals(IllegalStateException.class, ASYNC_CONSUMER_EXCEPTION.getClass()); - } finally { - if (flink != null) { - flink.stop(); - } } } From 555e5e8cd0a7b08457f65364ae62d13eb593fb44 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 20 Mar 2018 11:38:24 +0100 Subject: [PATCH 204/268] [hotfix][utils] Add ExceptionUtils#findThrowable with predicate --- .../org/apache/flink/util/ExceptionUtils.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java index 6af16fcfa4f6c..459648fdc8265 100644 --- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java @@ -35,6 +35,7 @@ import java.util.Optional; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; +import java.util.function.Predicate; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -325,6 +326,30 @@ public static Optional findThrowable(Throwable throwabl return Optional.empty(); } + /** + * Checks whether a throwable chain contains an exception matching a predicate and returns it. + * + * @param throwable the throwable chain to check. + * @param predicate the predicate of the exception to search for in the chain. + * @return Optional throwable of the requested type if available, otherwise empty + */ + public static Optional findThrowable(Throwable throwable, Predicate predicate) { + if (throwable == null || predicate == null) { + return Optional.empty(); + } + + Throwable t = throwable; + while (t != null) { + if (predicate.test(t)) { + return Optional.of(t); + } else { + t = t.getCause(); + } + } + + return Optional.empty(); + } + /** * Checks whether a throwable chain contains a specific error message and returns the corresponding throwable. * From f0bd7b67aa348597cda9c4d3cf920ffd5a320896 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 20 Mar 2018 11:40:48 +0100 Subject: [PATCH 205/268] [FLINK-8964][tests] Port JobSubmissionFailsITCase to flip6 This closes #5727. --- .../failing/JobSubmissionFailsITCase.java | 169 ++++++------------ 1 file changed, 55 insertions(+), 114 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java index a647af9c447c0..ecd16a1ff30b8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/example/failing/JobSubmissionFailsITCase.java @@ -19,30 +19,25 @@ package org.apache.flink.test.example.failing; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; -import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.test.util.MiniClusterResource; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.util.Arrays; import java.util.Collection; +import java.util.Optional; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -51,47 +46,32 @@ @RunWith(Parameterized.class) public class JobSubmissionFailsITCase extends TestLogger { + private static final int NUM_TM = 2; private static final int NUM_SLOTS = 20; - private static LocalFlinkMiniCluster cluster; - private static JobGraph workingJobGraph; - - @BeforeClass - public static void setup() { - try { - Configuration config = new Configuration(); - config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS / 2); - - cluster = new LocalFlinkMiniCluster(config); - - cluster.start(); - - final JobVertex jobVertex = new JobVertex("Working job vertex."); - jobVertex.setInvokableClass(NoOpInvokable.class); - workingJobGraph = new JobGraph("Working testing job", jobVertex); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + @ClassRule + public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfiguration(), + NUM_TM, + NUM_SLOTS / NUM_TM), + true); + + private static Configuration getConfiguration() { + Configuration config = new Configuration(); + config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L); + return config; } - @AfterClass - public static void teardown() { - try { - cluster.stop(); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + private static JobGraph getWorkingJobGraph() { + final JobVertex jobVertex = new JobVertex("Working job vertex."); + jobVertex.setInvokableClass(NoOpInvokable.class); + return new JobGraph("Working testing job", jobVertex); } // -------------------------------------------------------------------------------------------- - private boolean detached; + private final boolean detached; public JobSubmissionFailsITCase(boolean detached) { this.detached = detached; @@ -105,90 +85,51 @@ public static Collection executionModes(){ // -------------------------------------------------------------------------------------------- - private JobExecutionResult submitJob(JobGraph jobGraph) throws Exception { - if (detached) { - cluster.submitJobDetached(jobGraph); - return null; - } - else { - return cluster.submitJobAndWait(jobGraph, false, TestingUtils.TESTING_DURATION()); - } - } - @Test - public void testExceptionInInitializeOnMaster() { - try { - final JobVertex failingJobVertex = new FailingJobVertex("Failing job vertex"); - failingJobVertex.setInvokableClass(NoOpInvokable.class); - - final JobGraph failingJobGraph = new JobGraph("Failing testing job", failingJobVertex); + public void testExceptionInInitializeOnMaster() throws Exception { + final JobVertex failingJobVertex = new FailingJobVertex("Failing job vertex"); + failingJobVertex.setInvokableClass(NoOpInvokable.class); - try { - submitJob(failingJobGraph); - fail("Expected JobExecutionException."); - } - catch (JobExecutionException e) { - assertEquals("Test exception.", e.getCause().getMessage()); - } - catch (Throwable t) { - t.printStackTrace(); - fail("Caught wrong exception of type " + t.getClass() + "."); - } + final JobGraph failingJobGraph = new JobGraph("Failing testing job", failingJobVertex); - cluster.submitJobAndWait(workingJobGraph, false); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } - } + ClusterClient client = MINI_CLUSTER_RESOURCE.getClusterClient(); + client.setDetached(detached); - @Test - public void testSubmitEmptyJobGraph() { try { - final JobGraph jobGraph = new JobGraph("Testing job"); - - try { - submitJob(jobGraph); - fail("Expected JobSubmissionException."); - } - catch (JobSubmissionException e) { - assertTrue(e.getMessage() != null && e.getMessage().contains("empty")); + client.submitJob(failingJobGraph, JobSubmissionFailsITCase.class.getClassLoader()); + fail("Job submission should have thrown an exception."); + } catch (Exception e) { + Optional expectedCause = ExceptionUtils.findThrowable(e, + candidate -> "Test exception.".equals(candidate.getMessage())); + if (!expectedCause.isPresent()) { + throw e; } - catch (Throwable t) { - t.printStackTrace(); - fail("Caught wrong exception of type " + t.getClass() + "."); - } - - cluster.submitJobAndWait(workingJobGraph, false); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); } + + client.setDetached(false); + client.submitJob(getWorkingJobGraph(), JobSubmissionFailsITCase.class.getClassLoader()); } @Test - public void testSubmitNullJobGraph() { + public void testSubmitEmptyJobGraph() throws Exception { + final JobGraph jobGraph = new JobGraph("Testing job"); + + ClusterClient client = MINI_CLUSTER_RESOURCE.getClusterClient(); + client.setDetached(detached); + try { - try { - submitJob(null); - fail("Expected JobSubmissionException."); - } - catch (NullPointerException e) { - // yo! + client.submitJob(jobGraph, JobSubmissionFailsITCase.class.getClassLoader()); + fail("Job submission should have thrown an exception."); + } catch (Exception e) { + Optional expectedCause = ExceptionUtils.findThrowable(e, + throwable -> throwable.getMessage() != null && throwable.getMessage().contains("empty")); + if (!expectedCause.isPresent()) { + throw e; } - catch (Throwable t) { - t.printStackTrace(); - fail("Caught wrong exception of type " + t.getClass() + "."); - } - - cluster.submitJobAndWait(workingJobGraph, false); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); } + + client.setDetached(false); + client.submitJob(getWorkingJobGraph(), JobSubmissionFailsITCase.class.getClassLoader()); } // -------------------------------------------------------------------------------------------- From 81d809a5f0030b14e0b7128d298cd9904e474ebd Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 26 Feb 2018 17:19:15 +0100 Subject: [PATCH 206/268] [FLINK-8965][tests] Port TimestampITCase to flip6 This closes #5728. --- .../streaming/runtime/TimestampITCase.java | 73 ++++++++++--------- 1 file changed, 38 insertions(+), 35 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java index 5e08e8ae3d2cb..3b46c8259e51e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java @@ -24,11 +24,11 @@ import org.apache.flink.api.common.functions.StoppableFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.testutils.MultiShotLatch; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -45,17 +45,18 @@ import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.util.TestStreamEnvironment; +import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.util.TestLogger; -import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; -import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import java.util.stream.Collectors; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -75,34 +76,24 @@ public class TimestampITCase extends TestLogger { // this is used in some tests to synchronize static MultiShotLatch latch; - private static LocalFlinkMiniCluster cluster; + @ClassRule + public static final MiniClusterResource CLUSTER = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfiguration(), + NUM_TASK_MANAGERS, + NUM_TASK_SLOTS), + true); - @Before - public void setupLatch() { - // ensure that we get a fresh latch for each test - latch = new MultiShotLatch(); - } - - @BeforeClass - public static void startCluster() { + private static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TASK_MANAGERS); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS); config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L); - - cluster = new LocalFlinkMiniCluster(config, false); - - cluster.start(); - - TestStreamEnvironment.setAsContext(cluster, PARALLELISM); + return config; } - @AfterClass - public static void shutdownCluster() { - cluster.stop(); - cluster = null; - - TestStreamEnvironment.unsetAsContext(); + @Before + public void setupLatch() { + // ensure that we get a fresh latch for each test + latch = new MultiShotLatch(); } /** @@ -162,7 +153,8 @@ public void testWatermarkPropagation() throws Exception { public void testWatermarkPropagationNoFinalWatermarkOnStop() throws Exception { // for this test to work, we need to be sure that no other jobs are being executed - while (!cluster.getCurrentlyRunningJobsJava().isEmpty()) { + final ClusterClient clusterClient = CLUSTER.getClusterClient(); + while (!getRunningJobs(clusterClient).isEmpty()) { Thread.sleep(100); } @@ -185,14 +177,15 @@ public void testWatermarkPropagationNoFinalWatermarkOnStop() throws Exception { .transform("Custom Operator", BasicTypeInfo.INT_TYPE_INFO, new CustomOperator(true)) .addSink(new DiscardingSink()); - new Thread("stopper") { + Thread t = new Thread("stopper") { @Override public void run() { try { // try until we get the running jobs - List running; - while ((running = cluster.getCurrentlyRunningJobsJava()).isEmpty()) { + List running = getRunningJobs(clusterClient); + while (running.isEmpty()) { Thread.sleep(10); + running = getRunningJobs(clusterClient); } JobID id = running.get(0); @@ -200,7 +193,7 @@ public void run() { // send stop until the job is stopped do { try { - cluster.stopJob(id); + clusterClient.stop(id); } catch (Exception e) { if (e.getCause() instanceof IllegalStateException) { @@ -214,13 +207,14 @@ public void run() { } Thread.sleep(10); } - while (!cluster.getCurrentlyRunningJobsJava().isEmpty()); + while (!getRunningJobs(clusterClient).isEmpty()); } catch (Throwable t) { t.printStackTrace(); } } - }.start(); + }; + t.start(); env.execute(); @@ -246,6 +240,7 @@ public void run() { subtaskWatermarks.get(subtaskWatermarks.size() - 1)); } } + t.join(); } /** @@ -855,4 +850,12 @@ public void run(SourceContext ctx) throws Exception { @Override public void cancel() {} } + + private static List getRunningJobs(ClusterClient client) throws Exception { + Collection statusMessages = client.listJobs().get(); + return statusMessages.stream() + .filter(status -> !status.getJobState().isGloballyTerminalState()) + .map(JobStatusMessage::getJobId) + .collect(Collectors.toList()); + } } From 995983c03bf89abbaea69e34fa1ee1880cfbc6ba Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Fri, 23 Mar 2018 22:14:46 +0100 Subject: [PATCH 207/268] [hotfix] [table] Add Java deprecation annotation to TableEnvironment.sql(). --- .../scala/org/apache/flink/table/api/TableEnvironment.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala index 13e56567ac038..d6106bec69bd9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala @@ -574,9 +574,11 @@ abstract class TableEnvironment(val config: TableConfig) { * tEnv.sql(s"SELECT * FROM $table") * }}} * + * @deprecated Use sqlQuery() instead. * @param query The SQL query to evaluate. * @return The result of the query as Table. */ + @Deprecated @deprecated("Please use sqlQuery() instead.") def sql(query: String): Table = { sqlQuery(query) From 24fd61a41cd35a4e8b79251f6ce9d4890ebaced0 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 21 Mar 2018 16:25:37 +0800 Subject: [PATCH 208/268] [FLINK-8975] [test] Add Kafka events generator job for StateMachineExample --- .../statemachine/KafkaEventsGeneratorJob.java | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.java new file mode 100644 index 0000000000000..059b2c0240a88 --- /dev/null +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/KafkaEventsGeneratorJob.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.streaming.examples.statemachine; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; +import org.apache.flink.streaming.examples.statemachine.generator.EventsGeneratorSource; +import org.apache.flink.streaming.examples.statemachine.kafka.EventDeSerializer; + +/** + * Job to generate input events that are written to Kafka, for the {@link StateMachineExample} job. + */ +public class KafkaEventsGeneratorJob { + + public static void main(String[] args) throws Exception { + + final ParameterTool params = ParameterTool.fromArgs(args); + + double errorRate = params.getDouble("error-rate", 0.0); + int sleep = params.getInt("sleep", 1); + + String kafkaTopic = params.get("kafka-topic"); + String brokers = params.get("brokers", "localhost:9092"); + + System.out.printf("Generating events to Kafka with standalone source with error rate %f and sleep delay %s millis\n", errorRate, sleep); + System.out.println(); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env + .addSource(new EventsGeneratorSource(errorRate, sleep)) + .addSink(new FlinkKafkaProducer010<>(brokers, kafkaTopic, new EventDeSerializer())); + + // trigger program execution + env.execute("State machine example Kafka events generator job"); + } + +} From 7a848df43366a8b8877cc1174b268f97f9109e00 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 21 Mar 2018 16:32:51 +0800 Subject: [PATCH 209/268] [FLINK-8975] [test] Add resume from savepoint end-to-end test This closes #5733. --- .../run-pre-commit-tests.sh | 8 + flink-end-to-end-tests/test-scripts/common.sh | 22 +++ .../test-scripts/test_resume_savepoint.sh | 155 ++++++++++++++++++ .../statemachine/StateMachineExample.java | 2 +- 4 files changed, 186 insertions(+), 1 deletion(-) create mode 100755 flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh diff --git a/flink-end-to-end-tests/run-pre-commit-tests.sh b/flink-end-to-end-tests/run-pre-commit-tests.sh index 2c1810b91c8bf..4fd580cc26dfc 100755 --- a/flink-end-to-end-tests/run-pre-commit-tests.sh +++ b/flink-end-to-end-tests/run-pre-commit-tests.sh @@ -53,6 +53,14 @@ if [ $EXIT_CODE == 0 ]; then EXIT_CODE=$? fi +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Resuming Savepoint end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_resume_savepoint.sh + EXIT_CODE=$? +fi + if [ $EXIT_CODE == 0 ]; then printf "\n==============================================================================\n" printf "Running class loading end-to-end test\n" diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh index ef4856f561b54..d4b91266ac865 100644 --- a/flink-end-to-end-tests/test-scripts/common.sh +++ b/flink-end-to-end-tests/test-scripts/common.sh @@ -111,6 +111,28 @@ function stop_cluster { rm $FLINK_DIR/log/* } +function wait_job_running { + for i in {1..10}; do + JOB_LIST_RESULT=$("$FLINK_DIR"/bin/flink list | grep "$1") + + if [[ "$JOB_LIST_RESULT" == "" ]]; then + echo "Job ($1) is not yet running." + else + echo "Job ($1) is running." + break + fi + sleep 1 + done +} + +function take_savepoint { + "$FLINK_DIR"/bin/flink savepoint $1 $2 +} + +function cancel_job { + "$FLINK_DIR"/bin/flink cancel $1 +} + function check_result_hash { local name=$1 local outfile_prefix=$2 diff --git a/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh b/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh new file mode 100755 index 0000000000000..7108d9007330a --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh @@ -0,0 +1,155 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +source "$(dirname "$0")"/common.sh + +# get Kafka 0.10.0 +mkdir -p $TEST_DATA_DIR +if [ -z "$3" ]; then + # need to download Kafka because no Kafka was specified on the invocation + KAFKA_URL="https://archive.apache.org/dist/kafka/0.10.2.0/kafka_2.11-0.10.2.0.tgz" + echo "Downloading Kafka from $KAFKA_URL" + curl "$KAFKA_URL" > $TEST_DATA_DIR/kafka.tgz +else + echo "Using specified Kafka from $3" + cp $3 $TEST_DATA_DIR/kafka.tgz +fi + +tar xzf $TEST_DATA_DIR/kafka.tgz -C $TEST_DATA_DIR/ +KAFKA_DIR=$TEST_DATA_DIR/kafka_2.11-0.10.2.0 + +# fix kafka config +sed -i -e "s+^\(dataDir\s*=\s*\).*$+\1$TEST_DATA_DIR/zookeeper+" $KAFKA_DIR/config/zookeeper.properties +sed -i -e "s+^\(log\.dirs\s*=\s*\).*$+\1$TEST_DATA_DIR/kafka+" $KAFKA_DIR/config/server.properties +$KAFKA_DIR/bin/zookeeper-server-start.sh -daemon $KAFKA_DIR/config/zookeeper.properties +$KAFKA_DIR/bin/kafka-server-start.sh -daemon $KAFKA_DIR/config/server.properties + +# modify configuration to have 2 slots +cp $FLINK_DIR/conf/flink-conf.yaml $FLINK_DIR/conf/flink-conf.yaml.bak +sed -i -e 's/taskmanager.numberOfTaskSlots: 1/taskmanager.numberOfTaskSlots: 2/' $FLINK_DIR/conf/flink-conf.yaml + +# modify configuration to use SLF4J reporter; we will be using this to monitor the state machine progress +cp $FLINK_DIR/opt/flink-metrics-slf4j-*.jar $FLINK_DIR/lib/ +echo "metrics.reporter.slf4j.class: org.apache.flink.metrics.slf4j.Slf4jReporter" >> $FLINK_DIR/conf/flink-conf.yaml +echo "metrics.reporter.slf4j.interval: 1 SECONDS" >> $FLINK_DIR/conf/flink-conf.yaml + +start_cluster + +# make sure to stop Kafka and ZooKeeper at the end, as well as cleaning up the Flink cluster and our moodifications +function test_cleanup { + $KAFKA_DIR/bin/kafka-server-stop.sh + $KAFKA_DIR/bin/zookeeper-server-stop.sh + + # revert our modifications to the Flink distribution + rm $FLINK_DIR/conf/flink-conf.yaml + mv $FLINK_DIR/conf/flink-conf.yaml.bak $FLINK_DIR/conf/flink-conf.yaml + rm $FLINK_DIR/lib/flink-metrics-slf4j-*.jar + + # make sure to run regular cleanup as well + cleanup +} +trap test_cleanup INT +trap test_cleanup EXIT + +# zookeeper outputs the "Node does not exist" bit to stderr +while [[ $($KAFKA_DIR/bin/zookeeper-shell.sh localhost:2181 get /brokers/ids/0 2>&1) =~ .*Node\ does\ not\ exist.* ]]; do + echo "Waiting for broker..." + sleep 1 +done + +# create the required topic +$KAFKA_DIR/bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic test-input + +# run the state machine example job +STATE_MACHINE_JOB=$($FLINK_DIR/bin/flink run -d $FLINK_DIR/examples/streaming/StateMachineExample.jar \ + --kafka-topic test-input \ + | grep "Job has been submitted with JobID" | sed 's/.* //g') + +wait_job_running $STATE_MACHINE_JOB + +# then, run the events generator +EVENTS_GEN_JOB=$($FLINK_DIR/bin/flink run -d -c org.apache.flink.streaming.examples.statemachine.KafkaEventsGeneratorJob $FLINK_DIR/examples/streaming/StateMachineExample.jar \ + --kafka-topic test-input --sleep 15 \ + | grep "Job has been submitted with JobID" | sed 's/.* //g') + +wait_job_running $EVENTS_GEN_JOB + +function get_metric_state_machine_processed_records { + grep ".State machine job.Flat Map -> Sink: Print to Std. Out.0.numRecordsIn:" $FLINK_DIR/log/*taskexecutor*.log | sed 's/.* //g' | tail -1 +} + +function get_num_metric_samples { + grep ".State machine job.Flat Map -> Sink: Print to Std. Out.0.numRecordsIn:" $FLINK_DIR/log/*taskexecutor*.log | wc -l +} + +# monitor the numRecordsIn metric of the state machine operator; +# only proceed to savepoint when the operator has processed 200 records +while : ; do + NUM_RECORDS=$(get_metric_state_machine_processed_records) + + if [ -z $NUM_RECORDS ]; then + NUM_RECORDS=0 + fi + + if (( $NUM_RECORDS < 200 )); then + echo "Waiting for state machine job to process up to 200 records, current progress: $NUM_RECORDS records ..." + sleep 1 + else + break + fi +done + +# take a savepoint of the state machine job +SAVEPOINT_PATH=$(take_savepoint $STATE_MACHINE_JOB $TEST_DATA_DIR \ + | grep "Savepoint completed. Path:" | sed 's/.* //g') + +cancel_job $STATE_MACHINE_JOB + +# Since it is not possible to differentiate reporter output between the first and second execution, +# we remember the number of metrics sampled in the first execution so that they can be ignored in the following monitorings +OLD_NUM_METRICS=$(get_num_metric_samples) + +# resume state machine job with savepoint +STATE_MACHINE_JOB=$($FLINK_DIR/bin/flink run -s $SAVEPOINT_PATH -d $FLINK_DIR/examples/streaming/StateMachineExample.jar \ + --kafka-topic test-input \ + | grep "Job has been submitted with JobID" | sed 's/.* //g') + +wait_job_running $STATE_MACHINE_JOB + +# monitor the numRecordsIn metric of the state machine operator in the second execution +# we let the test finish once the second restore execution has processed 200 records +while : ; do + NUM_METRICS=$(get_num_metric_samples) + NUM_RECORDS=$(get_metric_state_machine_processed_records) + + # only account for metrics that appeared in the second execution + if (( $OLD_NUM_METRICS >= $NUM_METRICS )) ; then + NUM_RECORDS=0 + fi + + if (( $NUM_RECORDS < 200 )); then + echo "Waiting for state machine job to process up to 200 records, current progress: $NUM_RECORDS records ..." + sleep 1 + else + break + fi +done + +# if state is errorneous and the state machine job produces alerting state transitions, +# output would be non-empty and the test will not pass diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java index 052e954690d64..14757fb325e6f 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java @@ -108,7 +108,7 @@ public static void main(String[] args) throws Exception { alerts.print(); // trigger program execution - env.execute(); + env.execute("State machine job"); } // ------------------------------------------------------------------------ From 2995f909c881673d1c6f64e3080d9451acde28cf Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 22 Mar 2018 17:09:53 +0800 Subject: [PATCH 210/268] [FLINK-8976] [test] Add end-to-end tests for resuming savepoint with differrent parallelism This closes #5745. --- .../run-pre-commit-tests.sh | 20 +++++++++++++++-- .../test-scripts/test_resume_savepoint.sh | 22 +++++++++++++++---- 2 files changed, 36 insertions(+), 6 deletions(-) diff --git a/flink-end-to-end-tests/run-pre-commit-tests.sh b/flink-end-to-end-tests/run-pre-commit-tests.sh index 4fd580cc26dfc..1de66cdce35a3 100755 --- a/flink-end-to-end-tests/run-pre-commit-tests.sh +++ b/flink-end-to-end-tests/run-pre-commit-tests.sh @@ -55,9 +55,25 @@ fi if [ $EXIT_CODE == 0 ]; then printf "\n==============================================================================\n" - printf "Running Resuming Savepoint end-to-end test\n" + printf "Running Resuming Savepoint (no parallelism change) end-to-end test\n" printf "==============================================================================\n" - $END_TO_END_DIR/test-scripts/test_resume_savepoint.sh + $END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Resuming Savepoint (scale up) end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 + EXIT_CODE=$? +fi + +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Resuming Savepoint (scale down) end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 EXIT_CODE=$? fi diff --git a/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh b/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh index 7108d9007330a..83e0e5a08ce58 100755 --- a/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh +++ b/flink-end-to-end-tests/test-scripts/test_resume_savepoint.sh @@ -17,6 +17,11 @@ # limitations under the License. ################################################################################ +if [ -z $1 ] || [ -z $2 ]; then + echo "Usage: ./test_resume_savepoint.sh " + exit 1 +fi + source "$(dirname "$0")"/common.sh # get Kafka 0.10.0 @@ -40,9 +45,18 @@ sed -i -e "s+^\(log\.dirs\s*=\s*\).*$+\1$TEST_DATA_DIR/kafka+" $KAFKA_DIR/config $KAFKA_DIR/bin/zookeeper-server-start.sh -daemon $KAFKA_DIR/config/zookeeper.properties $KAFKA_DIR/bin/kafka-server-start.sh -daemon $KAFKA_DIR/config/server.properties -# modify configuration to have 2 slots +ORIGINAL_DOP=$1 +NEW_DOP=$2 + +if (( $ORIGINAL_DOP >= $NEW_DOP )); then + NUM_SLOTS=$(( $ORIGINAL_DOP + 1 )) +else + NUM_SLOTS=$(( $NEW_DOP + 1 )) +fi + +# modify configuration to have enough slots cp $FLINK_DIR/conf/flink-conf.yaml $FLINK_DIR/conf/flink-conf.yaml.bak -sed -i -e 's/taskmanager.numberOfTaskSlots: 1/taskmanager.numberOfTaskSlots: 2/' $FLINK_DIR/conf/flink-conf.yaml +sed -i -e "s/taskmanager.numberOfTaskSlots: 1/taskmanager.numberOfTaskSlots: $NUM_SLOTS/" $FLINK_DIR/conf/flink-conf.yaml # modify configuration to use SLF4J reporter; we will be using this to monitor the state machine progress cp $FLINK_DIR/opt/flink-metrics-slf4j-*.jar $FLINK_DIR/lib/ @@ -77,7 +91,7 @@ done $KAFKA_DIR/bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 --partitions 1 --topic test-input # run the state machine example job -STATE_MACHINE_JOB=$($FLINK_DIR/bin/flink run -d $FLINK_DIR/examples/streaming/StateMachineExample.jar \ +STATE_MACHINE_JOB=$($FLINK_DIR/bin/flink run -d -p $ORIGINAL_DOP $FLINK_DIR/examples/streaming/StateMachineExample.jar \ --kafka-topic test-input \ | grep "Job has been submitted with JobID" | sed 's/.* //g') @@ -126,7 +140,7 @@ cancel_job $STATE_MACHINE_JOB OLD_NUM_METRICS=$(get_num_metric_samples) # resume state machine job with savepoint -STATE_MACHINE_JOB=$($FLINK_DIR/bin/flink run -s $SAVEPOINT_PATH -d $FLINK_DIR/examples/streaming/StateMachineExample.jar \ +STATE_MACHINE_JOB=$($FLINK_DIR/bin/flink run -s $SAVEPOINT_PATH -p $NEW_DOP -d $FLINK_DIR/examples/streaming/StateMachineExample.jar \ --kafka-topic test-input \ | grep "Job has been submitted with JobID" | sed 's/.* //g') From ba4590db6afb0c8aa5c7aa984a88f716a8660996 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 21 Mar 2018 10:54:49 +0100 Subject: [PATCH 211/268] [hotfix][runtime] Remove unused method --- .../executiongraph/ArchivedExecutionGraph.java | 5 +---- .../runtime/executiongraph/ExecutionGraph.java | 15 --------------- 2 files changed, 1 insertion(+), 19 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java index d285b20ea1fed..d471f86ffd3c1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java @@ -229,10 +229,7 @@ public boolean isArchived() { return true; } - public StringifiedAccumulatorResult[] getUserAccumulators() { - return archivedUserAccumulators; - } - + @Override public ArchivedExecutionConfig getArchivedExecutionConfig() { return archivedExecutionConfig; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index ee23884d3a6f7..590fb075fe49a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -763,21 +763,6 @@ public Executor getFutureExecutor() { return userAccumulators; } - /** - * Gets the accumulator results. - */ - public Map getAccumulators() { - - Map> accumulatorMap = aggregateUserAccumulators(); - - Map result = new HashMap<>(); - for (Map.Entry> entry : accumulatorMap.entrySet()) { - result.put(entry.getKey(), entry.getValue().getLocalValue()); - } - - return result; - } - /** * Gets a serialized accumulator map. * @return The accumulator map with serialized accumulator values. From 58a589dcb88f7e48435b870caf88055f658d33df Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 22 Mar 2018 15:39:59 +0100 Subject: [PATCH 212/268] [hotfix][tests] Do not hide original exception in the tests --- .../SerializedJobExecutionResultTest.java | 106 ++++++++---------- 1 file changed, 47 insertions(+), 59 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java index b3bac5802b9a7..38447e2a201d3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java @@ -36,68 +36,56 @@ public class SerializedJobExecutionResultTest { @Test - public void testSerialization() { - try { - final ClassLoader classloader = getClass().getClassLoader(); - - JobID origJobId = new JobID(); - long origTime = 65927436589267L; - - Map> origMap = new HashMap>(); - origMap.put("name1", new SerializedValue(723L)); - origMap.put("name2", new SerializedValue("peter")); - - SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap); - - // serialize and deserialize the object - SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); - - assertEquals(origJobId, cloned.getJobId()); - assertEquals(origTime, cloned.getNetRuntime()); - assertEquals(origTime, cloned.getNetRuntime(TimeUnit.MILLISECONDS)); - assertEquals(origMap, cloned.getSerializedAccumulatorResults()); - - // convert to deserialized result - JobExecutionResult jResult = result.toJobExecutionResult(classloader); - JobExecutionResult jResultCopied = result.toJobExecutionResult(classloader); - - assertEquals(origJobId, jResult.getJobID()); - assertEquals(origJobId, jResultCopied.getJobID()); - assertEquals(origTime, jResult.getNetRuntime()); - assertEquals(origTime, jResult.getNetRuntime(TimeUnit.MILLISECONDS)); - assertEquals(origTime, jResultCopied.getNetRuntime()); - assertEquals(origTime, jResultCopied.getNetRuntime(TimeUnit.MILLISECONDS)); - - for (Map.Entry> entry : origMap.entrySet()) { - String name = entry.getKey(); - Object value = entry.getValue().deserializeValue(classloader); - assertEquals(value, jResult.getAccumulatorResult(name)); - assertEquals(value, jResultCopied.getAccumulatorResult(name)); - } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + public void testSerialization() throws Exception { + final ClassLoader classloader = getClass().getClassLoader(); + + JobID origJobId = new JobID(); + long origTime = 65927436589267L; + + Map> origMap = new HashMap>(); + origMap.put("name1", new SerializedValue(723L)); + origMap.put("name2", new SerializedValue("peter")); + + SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap); + + // serialize and deserialize the object + SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); + + assertEquals(origJobId, cloned.getJobId()); + assertEquals(origTime, cloned.getNetRuntime()); + assertEquals(origTime, cloned.getNetRuntime(TimeUnit.MILLISECONDS)); + assertEquals(origMap, cloned.getSerializedAccumulatorResults()); + + // convert to deserialized result + JobExecutionResult jResult = result.toJobExecutionResult(classloader); + JobExecutionResult jResultCopied = result.toJobExecutionResult(classloader); + + assertEquals(origJobId, jResult.getJobID()); + assertEquals(origJobId, jResultCopied.getJobID()); + assertEquals(origTime, jResult.getNetRuntime()); + assertEquals(origTime, jResult.getNetRuntime(TimeUnit.MILLISECONDS)); + assertEquals(origTime, jResultCopied.getNetRuntime()); + assertEquals(origTime, jResultCopied.getNetRuntime(TimeUnit.MILLISECONDS)); + + for (Map.Entry> entry : origMap.entrySet()) { + String name = entry.getKey(); + Object value = entry.getValue().deserializeValue(classloader); + assertEquals(value, jResult.getAccumulatorResult(name)); + assertEquals(value, jResultCopied.getAccumulatorResult(name)); } } @Test - public void testSerializationWithNullValues() { - try { - SerializedJobExecutionResult result = new SerializedJobExecutionResult(null, 0L, null); - SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); - - assertNull(cloned.getJobId()); - assertEquals(0L, cloned.getNetRuntime()); - assertNull(cloned.getSerializedAccumulatorResults()); - - JobExecutionResult jResult = result.toJobExecutionResult(getClass().getClassLoader()); - assertNull(jResult.getJobID()); - assertTrue(jResult.getAllAccumulatorResults().isEmpty()); - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } + public void testSerializationWithNullValues() throws Exception { + SerializedJobExecutionResult result = new SerializedJobExecutionResult(null, 0L, null); + SerializedJobExecutionResult cloned = CommonTestUtils.createCopySerializable(result); + + assertNull(cloned.getJobId()); + assertEquals(0L, cloned.getNetRuntime()); + assertNull(cloned.getSerializedAccumulatorResults()); + + JobExecutionResult jResult = result.toJobExecutionResult(getClass().getClassLoader()); + assertNull(jResult.getJobID()); + assertTrue(jResult.getAllAccumulatorResults().isEmpty()); } } From 18bcdd2ff65c88f37d59e6016f60239d38591b73 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 22 Mar 2018 17:55:49 +0100 Subject: [PATCH 213/268] [hotfix][tests] Allow to run SpillableSubpartitionTests in the loop --- .../io/network/partition/SpillableSubpartitionTest.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java index 840669e7c3fcc..15acd7c70b0b9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java @@ -34,6 +34,7 @@ import org.junit.AfterClass; import org.junit.Assert; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -74,7 +75,12 @@ public class SpillableSubpartitionTest extends SubpartitionTestBase { private static final ExecutorService executorService = Executors.newCachedThreadPool(); /** Asynchronous I/O manager. */ - private static final IOManager ioManager = new IOManagerAsync(); + private static IOManager ioManager; + + @BeforeClass + public static void setup() { + ioManager = new IOManagerAsync(); + } @AfterClass public static void shutdown() { From 7e15f9f63afe46668f1fb557a496210b6cce8006 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 21 Mar 2018 13:08:36 +0100 Subject: [PATCH 214/268] [FLINK-8721][flip6] Handle archiving failures for accumulators During archivization, wrap errors thrown by users' Accumulators into a OptionalFailure and do not fail the job because of that. This closes #5737. --- .../flink/client/program/ClusterClient.java | 7 +- .../client/program/MiniClusterClient.java | 11 +- .../program/rest/RestClusterClient.java | 5 +- .../program/rest/RestClusterClientTest.java | 11 +- .../flink/api/common/JobExecutionResult.java | 14 +- .../accumulators/AccumulatorHelper.java | 63 ++++++-- .../FailedAccumulatorSerialization.java | 73 --------- .../common/operators/CollectionExecutor.java | 3 +- .../apache/flink/util/OptionalFailure.java | 135 ++++++++++++++++ .../flink/util/function/CheckedSupplier.java | 12 ++ .../FailedAccumulatorSerializationTest.java | 89 ----------- .../accumulators/AccumulatorSnapshot.java | 2 +- .../StringifiedAccumulatorResult.java | 54 +++++-- .../client/SerializedJobExecutionResult.java | 13 +- .../executiongraph/AccessExecutionGraph.java | 3 +- .../ArchivedExecutionGraph.java | 10 +- .../runtime/executiongraph/Execution.java | 10 +- .../executiongraph/ExecutionGraph.java | 46 +++--- .../executiongraph/ExecutionJobVertex.java | 3 +- .../flink/runtime/jobmaster/JobResult.java | 11 +- .../handler/job/JobAccumulatorsHandler.java | 3 +- .../rest/messages/JobAccumulatorsInfo.java | 7 +- .../messages/json/JobResultDeserializer.java | 9 +- .../messages/json/JobResultSerializer.java | 7 +- .../accumulators/AccumulatorMessages.scala | 7 +- .../StringifiedAccumulatorResultTest.java | 30 +++- .../SerializedJobExecutionResultTest.java | 46 ++++-- .../ArchivedExecutionGraphTest.java | 12 +- ...ecutionAttemptAccumulatorsHandlerTest.java | 9 +- .../utils/ArchivedExecutionGraphBuilder.java | 7 +- .../JobExecutionResultResponseBodyTest.java | 3 +- .../TestingJobManagerMessages.scala | 3 +- .../accumulators/AccumulatorErrorITCase.java | 149 +++++++++++------- .../test/accumulators/AccumulatorITCase.java | 1 + .../accumulators/AccumulatorLiveITCase.java | 2 +- .../LegacyAccumulatorLiveITCase.java | 7 +- .../utils/SavepointMigrationTestBase.java | 11 +- 37 files changed, 524 insertions(+), 364 deletions(-) delete mode 100644 flink-core/src/main/java/org/apache/flink/api/common/accumulators/FailedAccumulatorSerialization.java create mode 100644 flink-core/src/main/java/org/apache/flink/util/OptionalFailure.java delete mode 100644 flink-core/src/test/java/org/apache/flink/api/common/accumulators/FailedAccumulatorSerializationTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index 166d9770b48bb..79a5383109581 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -65,6 +65,7 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -792,7 +793,7 @@ public CompletableFuture> listJobs() throws Excepti * @param jobID The job identifier of a job. * @return A Map containing the accumulator's name and its value. */ - public Map getAccumulators(JobID jobID) throws Exception { + public Map> getAccumulators(JobID jobID) throws Exception { return getAccumulators(jobID, ClassLoader.getSystemClassLoader()); } @@ -803,7 +804,7 @@ public Map getAccumulators(JobID jobID) throws Exception { * @param loader The class loader for deserializing the accumulator results. * @return A Map containing the accumulator's name and its value. */ - public Map getAccumulators(JobID jobID, ClassLoader loader) throws Exception { + public Map> getAccumulators(JobID jobID, ClassLoader loader) throws Exception { ActorGateway jobManagerGateway = getJobManagerGateway(); Future response; @@ -816,7 +817,7 @@ public Map getAccumulators(JobID jobID, ClassLoader loader) thro Object result = Await.result(response, timeout); if (result instanceof AccumulatorResultsFound) { - Map> serializedAccumulators = + Map>> serializedAccumulators = ((AccumulatorResultsFound) result).result(); return AccumulatorHelper.deserializeAccumulators(serializedAccumulators, loader); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index 276df62f7233d..44f6ef630d26a 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; import javax.annotation.Nonnull; @@ -132,16 +133,16 @@ public CompletableFuture> listJobs() throws Excepti } @Override - public Map getAccumulators(JobID jobID) throws Exception { + public Map> getAccumulators(JobID jobID) throws Exception { return getAccumulators(jobID, ClassLoader.getSystemClassLoader()); } @Override - public Map getAccumulators(JobID jobID, ClassLoader loader) throws Exception { + public Map> getAccumulators(JobID jobID, ClassLoader loader) throws Exception { AccessExecutionGraph executionGraph = guardWithSingleRetry(() -> miniCluster.getExecutionGraph(jobID), scheduledExecutor).get(); - Map> accumulatorsSerialized = executionGraph.getAccumulatorsSerialized(); - Map result = new HashMap<>(accumulatorsSerialized.size()); - for (Map.Entry> acc : accumulatorsSerialized.entrySet()) { + Map>> accumulatorsSerialized = executionGraph.getAccumulatorsSerialized(); + Map> result = new HashMap<>(accumulatorsSerialized.size()); + for (Map.Entry>> acc : accumulatorsSerialized.entrySet()) { result.put(acc.getKey(), acc.getValue().deserializeValue(loader)); } return result; diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index f3f196182011e..2e1ffb0229818 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -89,6 +89,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.CheckedSupplier; @@ -409,7 +410,7 @@ private CompletableFuture triggerSavepoint( } @Override - public Map getAccumulators(final JobID jobID, ClassLoader loader) throws Exception { + public Map> getAccumulators(final JobID jobID, ClassLoader loader) throws Exception { final JobAccumulatorsHeaders accumulatorsHeaders = JobAccumulatorsHeaders.getInstance(); final JobAccumulatorsMessageParameters accMsgParams = accumulatorsHeaders.getUnresolvedMessageParameters(); accMsgParams.jobPathParameter.resolve(jobID); @@ -420,7 +421,7 @@ public Map getAccumulators(final JobID jobID, ClassLoader loader accMsgParams ); - Map result = Collections.emptyMap(); + Map> result = Collections.emptyMap(); try { result = responseFuture.thenApply((JobAccumulatorsInfo accumulatorsInfo) -> { diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index e98ba436abf29..77a4113f59ed4 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -84,6 +84,7 @@ import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.testutils.category.Flip6; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; @@ -362,7 +363,7 @@ public void testSubmitJobAndWaitForExecutionResult() throws Exception { JobExecutionResultResponseBody.created(new JobResult.Builder() .jobId(jobId) .netRuntime(Long.MAX_VALUE) - .accumulatorResults(Collections.singletonMap("testName", new SerializedValue<>(1.0))) + .accumulatorResults(Collections.singletonMap("testName", new SerializedValue<>(OptionalFailure.of(1.0)))) .build()), JobExecutionResultResponseBody.created(new JobResult.Builder() .jobId(jobId) @@ -558,12 +559,12 @@ public void testGetAccumulators() throws Exception { JobID id = new JobID(); { - Map accumulators = restClusterClient.getAccumulators(id); + Map> accumulators = restClusterClient.getAccumulators(id); assertNotNull(accumulators); assertEquals(1, accumulators.size()); assertEquals(true, accumulators.containsKey("testKey")); - assertEquals("testValue", accumulators.get("testKey").toString()); + assertEquals("testValue", accumulators.get("testKey").get().toString()); } } } @@ -594,9 +595,9 @@ protected CompletableFuture handleRequest( userTaskAccumulators.add(new JobAccumulatorsInfo.UserTaskAccumulator("testName", "testType", "testValue")); if (includeSerializedValue) { - Map> serializedUserTaskAccumulators = new HashMap<>(1); + Map>> serializedUserTaskAccumulators = new HashMap<>(1); try { - serializedUserTaskAccumulators.put("testKey", new SerializedValue<>("testValue")); + serializedUserTaskAccumulators.put("testKey", new SerializedValue<>(OptionalFailure.of("testValue"))); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java index a200d123713c2..9e1a3a552b664 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/JobExecutionResult.java @@ -20,10 +20,12 @@ import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.OptionalFailure; import java.util.Collections; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * The result of a job execution. Gives access to the execution time of the job, @@ -34,7 +36,7 @@ public class JobExecutionResult extends JobSubmissionResult { private final long netRuntime; - private final Map accumulatorResults; + private final Map> accumulatorResults; /** * Creates a new JobExecutionResult. @@ -43,7 +45,7 @@ public class JobExecutionResult extends JobSubmissionResult { * @param netRuntime The net runtime of the job (excluding pre-flight phase like the optimizer) in milliseconds * @param accumulators A map of all accumulators produced by the job. */ - public JobExecutionResult(JobID jobID, long netRuntime, Map accumulators) { + public JobExecutionResult(JobID jobID, long netRuntime, Map> accumulators) { super(jobID); this.netRuntime = netRuntime; @@ -85,7 +87,7 @@ public long getNetRuntime(TimeUnit desiredUnit) { */ @SuppressWarnings("unchecked") public T getAccumulatorResult(String accumulatorName) { - return (T) this.accumulatorResults.get(accumulatorName); + return (T) this.accumulatorResults.get(accumulatorName).getUnchecked(); } /** @@ -95,7 +97,9 @@ public T getAccumulatorResult(String accumulatorName) { * @return A map containing all accumulators produced by the job. */ public Map getAllAccumulatorResults() { - return this.accumulatorResults; + return accumulatorResults.entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().getUnchecked())); } /** @@ -109,7 +113,7 @@ public Map getAllAccumulatorResults() { @Deprecated @PublicEvolving public Integer getIntCounterResult(String accumulatorName) { - Object result = this.accumulatorResults.get(accumulatorName); + Object result = this.accumulatorResults.get(accumulatorName).getUnchecked(); if (result == null) { return null; } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java index 78fb68bf6fa2e..9bc129929ca92 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/AccumulatorHelper.java @@ -19,20 +19,27 @@ package org.apache.flink.api.common.accumulators; import org.apache.flink.annotation.Internal; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.Serializable; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.function.Supplier; /** * Helper functions for the interaction with {@link Accumulator}. */ @Internal public class AccumulatorHelper { + private static final Logger LOG = LoggerFactory.getLogger(AccumulatorHelper.class); /** * Merge two collections of accumulators. The second will be merged into the @@ -44,19 +51,28 @@ public class AccumulatorHelper { * The collection of accumulators that will be merged into the * other */ - public static void mergeInto(Map> target, Map> toMerge) { + public static void mergeInto(Map>> target, Map> toMerge) { for (Map.Entry> otherEntry : toMerge.entrySet()) { - Accumulator ownAccumulator = target.get(otherEntry.getKey()); + OptionalFailure> ownAccumulator = target.get(otherEntry.getKey()); if (ownAccumulator == null) { // Create initial counter (copy!) - target.put(otherEntry.getKey(), otherEntry.getValue().clone()); + target.put( + otherEntry.getKey(), + wrapUnchecked(otherEntry.getKey(), () -> otherEntry.getValue().clone())); + } + else if (ownAccumulator.isFailure()) { + continue; } else { + Accumulator accumulator = ownAccumulator.getUnchecked(); // Both should have the same type - AccumulatorHelper.compareAccumulatorTypes(otherEntry.getKey(), - ownAccumulator.getClass(), otherEntry.getValue().getClass()); + compareAccumulatorTypes(otherEntry.getKey(), + accumulator.getClass(), otherEntry.getValue().getClass()); // Merge target counter with other counter - mergeSingle(ownAccumulator, otherEntry.getValue()); + + target.put( + otherEntry.getKey(), + wrapUnchecked(otherEntry.getKey(), () -> mergeSingle(accumulator, otherEntry.getValue().clone()))); } } } @@ -64,8 +80,8 @@ public static void mergeInto(Map> target, Map void mergeSingle(Accumulator target, - Accumulator toMerge) { + private static Accumulator mergeSingle(Accumulator target, + Accumulator toMerge) { @SuppressWarnings("unchecked") Accumulator typedTarget = (Accumulator) target; @@ -73,6 +89,8 @@ private static void mergeSingle(Accumulator ta Accumulator typedToMerge = (Accumulator) toMerge; typedTarget.merge(typedToMerge); + + return typedTarget; } /** @@ -106,14 +124,25 @@ public static void compareAccumulatorTypes( * Transform the Map with accumulators into a Map containing only the * results. */ - public static Map toResultMap(Map> accumulators) { - Map resultMap = new HashMap(); + public static Map> toResultMap(Map> accumulators) { + Map> resultMap = new HashMap<>(); for (Map.Entry> entry : accumulators.entrySet()) { - resultMap.put(entry.getKey(), entry.getValue().getLocalValue()); + resultMap.put(entry.getKey(), wrapUnchecked(entry.getKey(), () -> entry.getValue().getLocalValue())); } return resultMap; } + private static OptionalFailure wrapUnchecked(String name, Supplier supplier) { + return OptionalFailure.createFrom(() -> { + try { + return supplier.get(); + } catch (RuntimeException ex) { + LOG.error("Unexpected error while handling accumulator [" + name + "]", ex); + throw new FlinkException(ex); + } + }); + } + public static String getResultsFormatted(Map map) { StringBuilder builder = new StringBuilder(); for (Map.Entry entry : map.entrySet()) { @@ -152,19 +181,19 @@ public static String getResultsFormatted(Map map) { * @throws IOException * @throws ClassNotFoundException */ - public static Map deserializeAccumulators( - Map> serializedAccumulators, ClassLoader loader) - throws IOException, ClassNotFoundException { + public static Map> deserializeAccumulators( + Map>> serializedAccumulators, + ClassLoader loader) throws IOException, ClassNotFoundException { if (serializedAccumulators == null || serializedAccumulators.isEmpty()) { return Collections.emptyMap(); } - Map accumulators = new HashMap<>(serializedAccumulators.size()); + Map> accumulators = new HashMap<>(serializedAccumulators.size()); - for (Map.Entry> entry : serializedAccumulators.entrySet()) { + for (Map.Entry>> entry : serializedAccumulators.entrySet()) { - Object value = null; + OptionalFailure value = null; if (entry.getValue() != null) { value = entry.getValue().deserializeValue(loader); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/FailedAccumulatorSerialization.java b/flink-core/src/main/java/org/apache/flink/api/common/accumulators/FailedAccumulatorSerialization.java deleted file mode 100644 index b208b9ee26852..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/common/accumulators/FailedAccumulatorSerialization.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.api.common.accumulators; - -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.Preconditions; - -import java.io.Serializable; - -/** - * {@link Accumulator} implementation which indicates a serialization problem with the original - * accumulator. Accessing any of the {@link Accumulator} method will result in throwing the - * serialization exception. - * - * @param type of the value - * @param type of the accumulator result - */ -public class FailedAccumulatorSerialization implements Accumulator { - private static final long serialVersionUID = 6965908827065879760L; - - private final Throwable throwable; - - public FailedAccumulatorSerialization(Throwable throwable) { - this.throwable = Preconditions.checkNotNull(throwable); - } - - public Throwable getThrowable() { - return throwable; - } - - @Override - public void add(V value) { - ExceptionUtils.rethrow(throwable); - } - - @Override - public R getLocalValue() { - ExceptionUtils.rethrow(throwable); - return null; - } - - @Override - public void resetLocal() { - ExceptionUtils.rethrow(throwable); - } - - @Override - public void merge(Accumulator other) { - ExceptionUtils.rethrow(throwable); - } - - @Override - public Accumulator clone() { - ExceptionUtils.rethrow(throwable); - return null; - } -} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java index 07f48fc1675fe..55f3df7d31c0e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/CollectionExecutor.java @@ -50,6 +50,7 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.types.Value; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Visitor; import java.util.ArrayList; @@ -115,7 +116,7 @@ public JobExecutionResult execute(Plan program) throws Exception { } long endTime = System.currentTimeMillis(); - Map accumulatorResults = AccumulatorHelper.toResultMap(accumulators); + Map> accumulatorResults = AccumulatorHelper.toResultMap(accumulators); return new JobExecutionResult(null, endTime - startTime, accumulatorResults); } diff --git a/flink-core/src/main/java/org/apache/flink/util/OptionalFailure.java b/flink-core/src/main/java/org/apache/flink/util/OptionalFailure.java new file mode 100644 index 0000000000000..ace3cad93e402 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/util/OptionalFailure.java @@ -0,0 +1,135 @@ +/* + * 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.util; + +import org.apache.flink.util.function.CheckedSupplier; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Wrapper around an object representing either a success (with a given value) or a failure cause. + */ +public class OptionalFailure implements Serializable { + private static final long serialVersionUID = 1L; + + @Nullable + private transient T value; + + @Nullable + private Throwable failureCause; + + private OptionalFailure(@Nullable T value, @Nullable Throwable failureCause) { + this.value = value; + this.failureCause = failureCause; + } + + public static OptionalFailure of(T value) { + return new OptionalFailure<>(value, null); + } + + public static OptionalFailure ofFailure(Throwable failureCause) { + return new OptionalFailure<>(null, failureCause); + } + + /** + * @return wrapped {@link OptionalFailure} returned by {@code valueSupplier} or wrapped failure if + * {@code valueSupplier} has thrown an {@link Exception}. + */ + public static OptionalFailure createFrom(CheckedSupplier valueSupplier) { + try { + return of(valueSupplier.get()); + } catch (Exception ex) { + return ofFailure(ex); + } + } + + /** + * @return stored value or throw a {@link FlinkException} with {@code failureCause}. + */ + public T get() throws FlinkException { + if (value != null) { + return value; + } + checkNotNull(failureCause); + throw new FlinkException(failureCause); + } + + /** + * @return same as {@link #get()} but throws a {@link FlinkRuntimeException}. + */ + public T getUnchecked() throws FlinkRuntimeException { + if (value != null) { + return value; + } + checkNotNull(failureCause); + throw new FlinkRuntimeException(failureCause); + } + + public Throwable getFailureCause() { + return checkNotNull(failureCause); + } + + public boolean isFailure() { + return failureCause != null; + } + + @Override + public int hashCode() { + return Objects.hash(value, failureCause); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (obj == this) { + return true; + } + if (!(obj instanceof OptionalFailure)) { + return false; + } + OptionalFailure other = (OptionalFailure) obj; + return Objects.equals(value, other.value) && + Objects.equals(failureCause, other.failureCause); + } + + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(value); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + value = (T) stream.readObject(); + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{value=" + value + ", failureCause=" + failureCause + "}"; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/util/function/CheckedSupplier.java b/flink-core/src/main/java/org/apache/flink/util/function/CheckedSupplier.java index a0bcc1311856b..585d705f59676 100644 --- a/flink-core/src/main/java/org/apache/flink/util/function/CheckedSupplier.java +++ b/flink-core/src/main/java/org/apache/flink/util/function/CheckedSupplier.java @@ -18,6 +18,8 @@ package org.apache.flink.util.function; +import org.apache.flink.util.FlinkException; + import java.util.function.Supplier; /** @@ -36,4 +38,14 @@ static Supplier unchecked(CheckedSupplier checkedSupplier) { }; } + static CheckedSupplier checked(Supplier supplier) { + return () -> { + try { + return supplier.get(); + } + catch (RuntimeException e) { + throw new FlinkException(e); + } + }; + } } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/accumulators/FailedAccumulatorSerializationTest.java b/flink-core/src/test/java/org/apache/flink/api/common/accumulators/FailedAccumulatorSerializationTest.java deleted file mode 100644 index 7335d30dfc8c2..0000000000000 --- a/flink-core/src/test/java/org/apache/flink/api/common/accumulators/FailedAccumulatorSerializationTest.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.api.common.accumulators; - -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.InstantiationUtil; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.io.IOException; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; - -/** - * Tests for the {@link FailedAccumulatorSerialization}. - */ -public class FailedAccumulatorSerializationTest extends TestLogger { - - private static final IOException TEST_EXCEPTION = new IOException("Test exception"); - - /** - * Tests that any method call will throw the contained throwable (wrapped in an - * unchecked exception if it is checked). - */ - @Test - public void testMethodCallThrowsException() { - final FailedAccumulatorSerialization accumulator = new FailedAccumulatorSerialization<>(TEST_EXCEPTION); - - try { - accumulator.getLocalValue(); - } catch (RuntimeException re) { - assertThat(ExceptionUtils.findThrowableWithMessage(re, TEST_EXCEPTION.getMessage()).isPresent(), is(true)); - } - - try { - accumulator.resetLocal(); - } catch (RuntimeException re) { - assertThat(ExceptionUtils.findThrowableWithMessage(re, TEST_EXCEPTION.getMessage()).isPresent(), is(true)); - } - - try { - accumulator.add(1); - } catch (RuntimeException re) { - assertThat(ExceptionUtils.findThrowableWithMessage(re, TEST_EXCEPTION.getMessage()).isPresent(), is(true)); - } - - try { - accumulator.merge(new IntMinimum()); - } catch (RuntimeException re) { - assertThat(ExceptionUtils.findThrowableWithMessage(re, TEST_EXCEPTION.getMessage()).isPresent(), is(true)); - } - } - - /** - * Tests that the class can be serialized and deserialized using Java serialization. - */ - @Test - public void testSerialization() throws Exception { - final FailedAccumulatorSerialization accumulator = new FailedAccumulatorSerialization<>(TEST_EXCEPTION); - - final byte[] serializedAccumulator = InstantiationUtil.serializeObject(accumulator); - - final FailedAccumulatorSerialization deserializedAccumulator = InstantiationUtil.deserializeObject(serializedAccumulator, ClassLoader.getSystemClassLoader()); - - assertThat(deserializedAccumulator.getThrowable(), is(instanceOf(TEST_EXCEPTION.getClass()))); - assertThat(deserializedAccumulator.getThrowable().getMessage(), is(equalTo(TEST_EXCEPTION.getMessage()))); - } - -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java index 0bfb1acf27437..d01cd322fd42a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/AccumulatorSnapshot.java @@ -48,7 +48,7 @@ public AccumulatorSnapshot(JobID jobID, ExecutionAttemptID executionAttemptID, Map> userAccumulators) throws IOException { this.jobID = jobID; this.executionAttemptID = executionAttemptID; - this.userAccumulators = new SerializedValue>>(userAccumulators); + this.userAccumulators = new SerializedValue<>(userAccumulators); } public JobID getJobID() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java index b55159c00c55b..f283bcdcd72e4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResult.java @@ -19,6 +19,13 @@ package org.apache.flink.runtime.accumulators; import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.OptionalFailure; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; import java.util.Map; @@ -26,6 +33,7 @@ * Container class that transports the result of an accumulator as set of strings. */ public class StringifiedAccumulatorResult implements java.io.Serializable{ + private static final Logger LOG = LoggerFactory.getLogger(StringifiedAccumulatorResult.class); private static final long serialVersionUID = -4642311296836822611L; @@ -58,7 +66,7 @@ public String getValue() { /** * Flatten a map of accumulator names to Accumulator instances into an array of StringifiedAccumulatorResult values. */ - public static StringifiedAccumulatorResult[] stringifyAccumulatorResults(Map> accs) { + public static StringifiedAccumulatorResult[] stringifyAccumulatorResults(Map>> accs) { if (accs == null || accs.isEmpty()) { return new StringifiedAccumulatorResult[0]; } @@ -66,23 +74,37 @@ public static StringifiedAccumulatorResult[] stringifyAccumulatorResults(Map> entry : accs.entrySet()) { - StringifiedAccumulatorResult result; - Accumulator accumulator = entry.getValue(); - if (accumulator != null) { - Object localValue = accumulator.getLocalValue(); - if (localValue != null) { - result = new StringifiedAccumulatorResult(entry.getKey(), accumulator.getClass().getSimpleName(), localValue.toString()); - } else { - result = new StringifiedAccumulatorResult(entry.getKey(), accumulator.getClass().getSimpleName(), "null"); - } - } else { - result = new StringifiedAccumulatorResult(entry.getKey(), "null", "null"); - } - - results[i++] = result; + for (Map.Entry>> entry : accs.entrySet()) { + results[i++] = stringifyAccumulatorResult(entry.getKey(), entry.getValue()); } return results; } } + + private static StringifiedAccumulatorResult stringifyAccumulatorResult( + String name, + @Nullable OptionalFailure> accumulator) { + if (accumulator == null) { + return new StringifiedAccumulatorResult(name, "null", "null"); + } + else if (accumulator.isFailure()) { + return new StringifiedAccumulatorResult( + name, + "null", + ExceptionUtils.stringifyException(accumulator.getFailureCause())); + } + else { + Object localValue; + String simpleName = "null"; + try { + simpleName = accumulator.getUnchecked().getClass().getSimpleName(); + localValue = accumulator.getUnchecked().getLocalValue(); + } + catch (RuntimeException exception) { + LOG.error("Failed to stringify accumulator [" + name + "]", exception); + localValue = ExceptionUtils.stringifyException(exception); + } + return new StringifiedAccumulatorResult(name, simpleName, localValue != null ? localValue.toString() : "null"); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java index ec2312f36b878..0ca7a4145da54 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/SerializedJobExecutionResult.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.AccumulatorHelper; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; import java.io.IOException; @@ -37,7 +38,7 @@ public class SerializedJobExecutionResult implements java.io.Serializable { private final JobID jobId; - private final Map> accumulatorResults; + private final Map>> accumulatorResults; private final long netRuntime; @@ -48,8 +49,10 @@ public class SerializedJobExecutionResult implements java.io.Serializable { * @param netRuntime The net runtime of the job (excluding pre-flight phase like the optimizer) in milliseconds * @param accumulators A map of all accumulator results produced by the job, in serialized form */ - public SerializedJobExecutionResult(JobID jobID, long netRuntime, - Map> accumulators) { + public SerializedJobExecutionResult( + JobID jobID, + long netRuntime, + Map>> accumulators) { this.jobId = jobID; this.netRuntime = netRuntime; this.accumulatorResults = accumulators; @@ -74,12 +77,12 @@ public long getNetRuntime(TimeUnit desiredUnit) { return desiredUnit.convert(getNetRuntime(), TimeUnit.MILLISECONDS); } - public Map> getSerializedAccumulatorResults() { + public Map>> getSerializedAccumulatorResults() { return this.accumulatorResults; } public JobExecutionResult toJobExecutionResult(ClassLoader loader) throws IOException, ClassNotFoundException { - Map accumulators = + Map> accumulators = AccumulatorHelper.deserializeAccumulators(accumulatorResults, loader); return new JobExecutionResult(jobId, netRuntime, accumulators); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java index cc56209e3de09..de578bdb9053b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AccessExecutionGraph.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; import javax.annotation.Nullable; @@ -155,7 +156,7 @@ public interface AccessExecutionGraph { * * @return map containing serialized values of user-defined accumulators */ - Map> getAccumulatorsSerialized(); + Map>> getAccumulatorsSerialized(); /** * Returns whether this execution graph was archived. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java index d471f86ffd3c1..24897215335fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -86,7 +87,7 @@ public class ArchivedExecutionGraph implements AccessExecutionGraph, Serializabl private final StringifiedAccumulatorResult[] archivedUserAccumulators; private final ArchivedExecutionConfig archivedExecutionConfig; private final boolean isStoppable; - private final Map> serializedUserAccumulators; + private final Map>> serializedUserAccumulators; @Nullable private final CheckpointCoordinatorConfiguration jobCheckpointingConfiguration; @@ -104,7 +105,7 @@ public ArchivedExecutionGraph( @Nullable ErrorInfo failureCause, String jsonPlan, StringifiedAccumulatorResult[] archivedUserAccumulators, - Map> serializedUserAccumulators, + Map>> serializedUserAccumulators, ArchivedExecutionConfig executionConfig, boolean isStoppable, @Nullable CheckpointCoordinatorConfiguration jobCheckpointingConfiguration, @@ -245,7 +246,7 @@ public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { } @Override - public Map> getAccumulatorsSerialized() { + public Map>> getAccumulatorsSerialized() { return serializedUserAccumulators; } @@ -312,7 +313,8 @@ public static ArchivedExecutionGraph createFrom(ExecutionGraph executionGraph) { archivedTasks.put(task.getJobVertexId(), archivedTask); } - final Map> serializedUserAccumulators = executionGraph.getAccumulatorsSerialized(); + final Map>> serializedUserAccumulators = + executionGraph.getAccumulatorsSerialized(); final long[] timestamps = new long[JobStatus.values().length]; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 2fb831a2c62b8..1af4fc99f9ec7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -53,6 +53,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.OptionalFailure; import org.slf4j.Logger; @@ -69,6 +70,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.stream.Collectors; import static org.apache.flink.runtime.execution.ExecutionState.CANCELED; import static org.apache.flink.runtime.execution.ExecutionState.CANCELING; @@ -1366,7 +1368,13 @@ public void setAccumulators(Map> userAccumulators) { @Override public StringifiedAccumulatorResult[] getUserAccumulatorsStringified() { - return StringifiedAccumulatorResult.stringifyAccumulatorResults(userAccumulators); + Map>> accumulators = + userAccumulators == null ? + null : + userAccumulators.entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> OptionalFailure.of(entry.getValue()))); + return StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulators); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java index 590fb075fe49a..22c11efae873b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.AccumulatorHelper; -import org.apache.flink.api.common.accumulators.FailedAccumulatorSerialization; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.JobException; @@ -69,6 +68,7 @@ import org.apache.flink.types.Either; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.SerializedValue; @@ -103,6 +103,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -749,9 +750,9 @@ public Executor getFutureExecutor() { * Merges all accumulator results from the tasks previously executed in the Executions. * @return The accumulator map */ - public Map> aggregateUserAccumulators() { + public Map>> aggregateUserAccumulators() { - Map> userAccumulators = new HashMap<>(); + Map>> userAccumulators = new HashMap<>(); for (ExecutionVertex vertex : getAllExecutionVertices()) { Map> next = vertex.getCurrentExecutionAttempt().getUserAccumulators(); @@ -768,28 +769,29 @@ public Executor getFutureExecutor() { * @return The accumulator map with serialized accumulator values. */ @Override - public Map> getAccumulatorsSerialized() { - - Map> accumulatorMap = aggregateUserAccumulators(); - - Map> result = new HashMap<>(accumulatorMap.size()); - for (Map.Entry> entry : accumulatorMap.entrySet()) { + public Map>> getAccumulatorsSerialized() { + return aggregateUserAccumulators() + .entrySet() + .stream() + .collect(Collectors.toMap( + Map.Entry::getKey, + entry -> serializeAccumulator(entry.getKey(), entry.getValue()))); + } + private static SerializedValue> serializeAccumulator(String name, OptionalFailure> accumulator) { + try { + if (accumulator.isFailure()) { + return new SerializedValue<>(OptionalFailure.ofFailure(accumulator.getFailureCause())); + } + return new SerializedValue<>(OptionalFailure.of(accumulator.getUnchecked().getLocalValue())); + } catch (IOException ioe) { + LOG.error("Could not serialize accumulator " + name + '.', ioe); try { - final SerializedValue serializedValue = new SerializedValue<>(entry.getValue().getLocalValue()); - result.put(entry.getKey(), serializedValue); - } catch (IOException ioe) { - LOG.error("Could not serialize accumulator " + entry.getKey() + '.', ioe); - - try { - result.put(entry.getKey(), new SerializedValue<>(new FailedAccumulatorSerialization(ioe))); - } catch (IOException e) { - throw new RuntimeException("It should never happen that we cannot serialize the accumulator serialization exception.", e); - } + return new SerializedValue<>(OptionalFailure.ofFailure(ioe)); + } catch (IOException e) { + throw new RuntimeException("It should never happen that we cannot serialize the accumulator serialization exception.", e); } } - - return result; } /** @@ -798,7 +800,7 @@ public Map> getAccumulatorsSerialized() { */ @Override public StringifiedAccumulatorResult[] getAccumulatorResultsStringified() { - Map> accumulatorMap = aggregateUserAccumulators(); + Map>> accumulatorMap = aggregateUserAccumulators(); return StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index 6e578fa0bda95..e5b7aa5d5bd9b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -48,6 +48,7 @@ import org.apache.flink.runtime.jobmaster.slotpool.SlotProvider; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.types.Either; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -597,7 +598,7 @@ public void resetForNewExecution(final long timestamp, final long expectedGlobal // -------------------------------------------------------------------------------------------- public StringifiedAccumulatorResult[] getAggregatedUserAccumulatorsStringified() { - Map> userAccumulators = new HashMap>(); + Map>> userAccumulators = new HashMap<>(); for (ExecutionVertex vertex : taskVertices) { Map> next = vertex.getCurrentExecutionAttempt().getUserAccumulators(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobResult.java index 28fbc3002a70e..76884046791ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobResult.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.SerializedValue; @@ -53,7 +54,7 @@ public class JobResult implements Serializable { private final JobID jobId; - private final Map> accumulatorResults; + private final Map>> accumulatorResults; private final long netRuntime; @@ -63,7 +64,7 @@ public class JobResult implements Serializable { private JobResult( final JobID jobId, - final Map> accumulatorResults, + final Map>> accumulatorResults, final long netRuntime, @Nullable final SerializedThrowable serializedThrowable) { @@ -86,7 +87,7 @@ public JobID getJobId() { return jobId; } - public Map> getAccumulatorResults() { + public Map>> getAccumulatorResults() { return accumulatorResults; } @@ -133,7 +134,7 @@ public static class Builder { private JobID jobId; - private Map> accumulatorResults; + private Map>> accumulatorResults; private long netRuntime = -1; @@ -144,7 +145,7 @@ public Builder jobId(final JobID jobId) { return this; } - public Builder accumulatorResults(final Map> accumulatorResults) { + public Builder accumulatorResults(final Map>> accumulatorResults) { this.accumulatorResults = accumulatorResults; return this; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobAccumulatorsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobAccumulatorsHandler.java index 0fe920171dcfa..964aee3bb91a7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobAccumulatorsHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/JobAccumulatorsHandler.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; import java.util.ArrayList; @@ -87,7 +88,7 @@ protected JobAccumulatorsInfo handleRequest(HandlerRequest> serializedUserTaskAccumulators = graph.getAccumulatorsSerialized(); + Map>> serializedUserTaskAccumulators = graph.getAccumulatorsSerialized(); accumulatorsInfo = new JobAccumulatorsInfo(Collections.emptyList(), userTaskAccumulators, serializedUserTaskAccumulators); } else { accumulatorsInfo = new JobAccumulatorsInfo(Collections.emptyList(), userTaskAccumulators, Collections.emptyMap()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfo.java index 22621204a7f9a..192eaa0246b1c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobAccumulatorsInfo.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.rest.handler.job.JobAccumulatorsHandler; import org.apache.flink.runtime.rest.messages.json.SerializedValueDeserializer; import org.apache.flink.runtime.rest.messages.json.SerializedValueSerializer; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -50,13 +51,13 @@ public class JobAccumulatorsInfo implements ResponseBody { @JsonProperty(FIELD_NAME_SERIALIZED_USER_TASK_ACCUMULATORS) @JsonSerialize(contentUsing = SerializedValueSerializer.class) - private Map> serializedUserAccumulators; + private Map>> serializedUserAccumulators; @JsonCreator public JobAccumulatorsInfo( @JsonProperty(FIELD_NAME_JOB_ACCUMULATORS) List jobAccumulators, @JsonProperty(FIELD_NAME_USER_TASK_ACCUMULATORS) List userAccumulators, - @JsonDeserialize(contentUsing = SerializedValueDeserializer.class) @JsonProperty(FIELD_NAME_SERIALIZED_USER_TASK_ACCUMULATORS) Map> serializedUserAccumulators) { + @JsonDeserialize(contentUsing = SerializedValueDeserializer.class) @JsonProperty(FIELD_NAME_SERIALIZED_USER_TASK_ACCUMULATORS) Map>> serializedUserAccumulators) { this.jobAccumulators = Preconditions.checkNotNull(jobAccumulators); this.userAccumulators = Preconditions.checkNotNull(userAccumulators); this.serializedUserAccumulators = Preconditions.checkNotNull(serializedUserAccumulators); @@ -73,7 +74,7 @@ public List getUserAccumulators() { } @JsonIgnore - public Map> getSerializedUserAccumulators() { + public Map>> getSerializedUserAccumulators() { return serializedUserAccumulators; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/JobResultDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/JobResultDeserializer.java index 52bb43cf69004..e568f476c7ed1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/JobResultDeserializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/JobResultDeserializer.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.SerializedValue; @@ -69,7 +70,7 @@ public JobResult deserialize(final JsonParser p, final DeserializationContext ct JobID jobId = null; long netRuntime = -1; SerializedThrowable serializedThrowable = null; - Map> accumulatorResults = null; + Map>> accumulatorResults = null; while (true) { final JsonToken jsonToken = p.nextToken(); @@ -117,11 +118,11 @@ public JobResult deserialize(final JsonParser p, final DeserializationContext ct } @SuppressWarnings("unchecked") - private Map> parseAccumulatorResults( + private Map>> parseAccumulatorResults( final JsonParser p, final DeserializationContext ctxt) throws IOException { - final Map> accumulatorResults = new HashMap<>(); + final Map>> accumulatorResults = new HashMap<>(); while (true) { final JsonToken jsonToken = p.nextToken(); assertNotEndOfInput(p, jsonToken); @@ -132,7 +133,7 @@ private Map> parseAccumulatorResults( p.nextValue(); accumulatorResults.put( accumulatorName, - (SerializedValue) serializedValueDeserializer.deserialize(p, ctxt)); + (SerializedValue>) serializedValueDeserializer.deserialize(p, ctxt)); } return accumulatorResults; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/JobResultSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/JobResultSerializer.java index a53716aab5691..694fa2f529bff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/JobResultSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/JobResultSerializer.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.rest.messages.json; import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.SerializedValue; @@ -77,10 +78,10 @@ public void serialize( gen.writeFieldName(FIELD_NAME_ACCUMULATOR_RESULTS); gen.writeStartObject(); - final Map> accumulatorResults = result.getAccumulatorResults(); - for (final Map.Entry> nameValue : accumulatorResults.entrySet()) { + final Map>> accumulatorResults = result.getAccumulatorResults(); + for (final Map.Entry>> nameValue : accumulatorResults.entrySet()) { final String name = nameValue.getKey(); - final SerializedValue value = nameValue.getValue(); + final SerializedValue> value = nameValue.getValue(); gen.writeFieldName(name); serializedValueSerializer.serialize(value, gen, provider); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/accumulators/AccumulatorMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/accumulators/AccumulatorMessages.scala index 107ba825864c6..9ed01aa350e66 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/accumulators/AccumulatorMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/accumulators/AccumulatorMessages.scala @@ -20,7 +20,7 @@ package org.apache.flink.runtime.messages.accumulators import org.apache.flink.api.common.JobID import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult -import org.apache.flink.util.SerializedValue +import org.apache.flink.util.{OptionalFailure, SerializedValue} /** * Base trait of all accumulator messages @@ -62,8 +62,9 @@ case class RequestAccumulatorResultsStringified(jobID: JobID) * @param jobID Job Id of the job that the accumulator belongs to * @param result The accumulator result values, in serialized form. */ -case class AccumulatorResultsFound(jobID: JobID, - result: java.util.Map[String, SerializedValue[Object]]) +case class AccumulatorResultsFound( + jobID: JobID, + result: java.util.Map[String, SerializedValue[OptionalFailure[Object]]]) extends AccumulatorResultsResponse /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResultTest.java index 65c4b7a39d53a..a06bdfce2af67 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/accumulators/StringifiedAccumulatorResultTest.java @@ -22,6 +22,8 @@ import org.apache.flink.api.common.accumulators.IntCounter; import org.apache.flink.api.common.accumulators.SimpleAccumulator; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.OptionalFailure; import org.junit.Test; @@ -31,6 +33,7 @@ import java.util.Map; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * Tests for the {@link StringifiedAccumulatorResult}. @@ -63,8 +66,8 @@ public void stringifyingResultsShouldIncorporateAccumulatorLocalValueDirectly() final int targetValue = 314159; final IntCounter acc = new IntCounter(); acc.add(targetValue); - final Map> accumulatorMap = new HashMap<>(); - accumulatorMap.put(name, acc); + final Map>> accumulatorMap = new HashMap<>(); + accumulatorMap.put(name, OptionalFailure.of(acc)); final StringifiedAccumulatorResult[] results = StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap); @@ -80,8 +83,8 @@ public void stringifyingResultsShouldIncorporateAccumulatorLocalValueDirectly() public void stringifyingResultsShouldReportNullLocalValueAsNonnullValueString() { final String name = "a"; final NullBearingAccumulator acc = new NullBearingAccumulator(); - final Map> accumulatorMap = new HashMap<>(); - accumulatorMap.put(name, acc); + final Map>> accumulatorMap = new HashMap<>(); + accumulatorMap.put(name, OptionalFailure.of(acc)); final StringifiedAccumulatorResult[] results = StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap); @@ -97,7 +100,7 @@ public void stringifyingResultsShouldReportNullLocalValueAsNonnullValueString() @Test public void stringifyingResultsShouldReportNullAccumulatorWithNonnullValueAndTypeString() { final String name = "a"; - final Map> accumulatorMap = new HashMap<>(); + final Map>> accumulatorMap = new HashMap<>(); accumulatorMap.put(name, null); final StringifiedAccumulatorResult[] results = StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap); @@ -111,6 +114,23 @@ public void stringifyingResultsShouldReportNullAccumulatorWithNonnullValueAndTyp assertEquals("null", firstResult.getValue()); } + @Test + public void stringifyingFailureResults() { + final String name = "a"; + final Map>> accumulatorMap = new HashMap<>(); + accumulatorMap.put(name, OptionalFailure.ofFailure(new FlinkRuntimeException("Test"))); + + final StringifiedAccumulatorResult[] results = StringifiedAccumulatorResult.stringifyAccumulatorResults(accumulatorMap); + + assertEquals(1, results.length); + + // Note the use of String values with content of "null" rather than null values + final StringifiedAccumulatorResult firstResult = results[0]; + assertEquals(name, firstResult.getName()); + assertEquals("null", firstResult.getType()); + assertTrue(firstResult.getValue().startsWith("org.apache.flink.util.FlinkRuntimeException: Test")); + } + private static class NullBearingAccumulator implements SimpleAccumulator { @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java index 38447e2a201d3..c16acec88f667 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/SerializedJobExecutionResultTest.java @@ -21,19 +21,28 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.operators.testutils.ExpectedTestException; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLogger; + import org.junit.Test; import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; /** * Tests for the SerializedJobExecutionResult */ -public class SerializedJobExecutionResultTest { +public class SerializedJobExecutionResultTest extends TestLogger { @Test public void testSerialization() throws Exception { @@ -42,9 +51,10 @@ public void testSerialization() throws Exception { JobID origJobId = new JobID(); long origTime = 65927436589267L; - Map> origMap = new HashMap>(); - origMap.put("name1", new SerializedValue(723L)); - origMap.put("name2", new SerializedValue("peter")); + Map>> origMap = new HashMap<>(); + origMap.put("name1", new SerializedValue<>(OptionalFailure.of(723L))); + origMap.put("name2", new SerializedValue<>(OptionalFailure.of("peter"))); + origMap.put("name3", new SerializedValue<>(OptionalFailure.ofFailure(new ExpectedTestException()))); SerializedJobExecutionResult result = new SerializedJobExecutionResult(origJobId, origTime, origMap); @@ -67,11 +77,29 @@ public void testSerialization() throws Exception { assertEquals(origTime, jResultCopied.getNetRuntime()); assertEquals(origTime, jResultCopied.getNetRuntime(TimeUnit.MILLISECONDS)); - for (Map.Entry> entry : origMap.entrySet()) { + for (Map.Entry>> entry : origMap.entrySet()) { String name = entry.getKey(); - Object value = entry.getValue().deserializeValue(classloader); - assertEquals(value, jResult.getAccumulatorResult(name)); - assertEquals(value, jResultCopied.getAccumulatorResult(name)); + OptionalFailure value = entry.getValue().deserializeValue(classloader); + if (value.isFailure()) { + try { + jResult.getAccumulatorResult(name); + fail("expected failure"); + } + catch (FlinkRuntimeException ex) { + assertTrue(ExceptionUtils.findThrowable(ex, ExpectedTestException.class).isPresent()); + } + try { + jResultCopied.getAccumulatorResult(name); + fail("expected failure"); + } + catch (FlinkRuntimeException ex) { + assertTrue(ExceptionUtils.findThrowable(ex, ExpectedTestException.class).isPresent()); + } + } + else { + assertEquals(value.get(), jResult.getAccumulatorResult(name)); + assertEquals(value.get(), jResultCopied.getAccumulatorResult(name)); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java index f15dca1fcc9c6..7c2a02c3314f2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java @@ -44,6 +44,7 @@ import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; @@ -58,6 +59,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -310,11 +312,13 @@ private static void compareStringifiedAccumulators(StringifiedAccumulatorResult[ } } - private static void compareSerializedAccumulators(Map> runtimeAccs, Map> archivedAccs) throws IOException, ClassNotFoundException { + private static void compareSerializedAccumulators( + Map>> runtimeAccs, + Map>> archivedAccs) throws IOException, ClassNotFoundException { assertEquals(runtimeAccs.size(), archivedAccs.size()); - for (Map.Entry> runtimeAcc : runtimeAccs.entrySet()) { - long runtimeUserAcc = (long) runtimeAcc.getValue().deserializeValue(ClassLoader.getSystemClassLoader()); - long archivedUserAcc = (long) archivedAccs.get(runtimeAcc.getKey()).deserializeValue(ClassLoader.getSystemClassLoader()); + for (Entry>> runtimeAcc : runtimeAccs.entrySet()) { + long runtimeUserAcc = (long) runtimeAcc.getValue().deserializeValue(ClassLoader.getSystemClassLoader()).getUnchecked(); + long archivedUserAcc = (long) archivedAccs.get(runtimeAcc.getKey()).deserializeValue(ClassLoader.getSystemClassLoader()).getUnchecked(); assertEquals(runtimeUserAcc, archivedUserAcc); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java index 318541d288612..df4ff04bf7427 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java @@ -36,6 +36,8 @@ import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptAccumulatorsInfo; import org.apache.flink.runtime.rest.messages.job.UserAccumulator; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -76,9 +78,10 @@ public void testHandleRequest() throws Exception { new SubtaskAttemptMessageParameters() ); - final Map> userAccumulators = new HashMap<>(2); - userAccumulators.put("IntCounter", new IntCounter(10)); - userAccumulators.put("LongCounter", new LongCounter(100L)); + final Map>> userAccumulators = new HashMap<>(3); + userAccumulators.put("IntCounter", OptionalFailure.of(new IntCounter(10))); + userAccumulators.put("LongCounter", OptionalFailure.of(new LongCounter(100L))); + userAccumulators.put("Failure", OptionalFailure.ofFailure(new FlinkRuntimeException("Test"))); // Instance the expected result. final StringifiedAccumulatorResult[] accumulatorResults = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionGraphBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionGraphBuilder.java index ee7ceda1400bc..689e1b316d279 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionGraphBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/utils/ArchivedExecutionGraphBuilder.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -53,7 +54,7 @@ public class ArchivedExecutionGraphBuilder { private StringifiedAccumulatorResult[] archivedUserAccumulators; private ArchivedExecutionConfig archivedExecutionConfig; private boolean isStoppable; - private Map> serializedUserAccumulators; + private Map>> serializedUserAccumulators; public ArchivedExecutionGraphBuilder setJobID(JobID jobID) { this.jobID = jobID; @@ -111,7 +112,7 @@ public ArchivedExecutionGraphBuilder setStoppable(boolean stoppable) { return this; } - public ArchivedExecutionGraphBuilder setSerializedUserAccumulators(Map> serializedUserAccumulators) { + public ArchivedExecutionGraphBuilder setSerializedUserAccumulators(Map>> serializedUserAccumulators) { this.serializedUserAccumulators = serializedUserAccumulators; return this; } @@ -134,7 +135,7 @@ public ArchivedExecutionGraph build() { failureCause, jsonPlan != null ? jsonPlan : "{\"jobid\":\"" + jobID + "\", \"name\":\"" + jobName + "\", \"nodes\":[]}", archivedUserAccumulators != null ? archivedUserAccumulators : new StringifiedAccumulatorResult[0], - serializedUserAccumulators != null ? serializedUserAccumulators : Collections.>emptyMap(), + serializedUserAccumulators != null ? serializedUserAccumulators : Collections.emptyMap(), archivedExecutionConfig != null ? archivedExecutionConfig : new ArchivedExecutionConfigBuilder().build(), isStoppable, null, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobExecutionResultResponseBodyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobExecutionResultResponseBodyTest.java index d7dd7eb725c7c..9534d2bca7cc6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobExecutionResultResponseBodyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/JobExecutionResultResponseBodyTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rest.messages.RestResponseMarshallingTestBase; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.SerializedValue; @@ -54,7 +55,7 @@ public class JobExecutionResultResponseBodyTest private static final String TEST_ACCUMULATOR_NAME = "test"; - private static final Map> TEST_ACCUMULATORS = Collections.singletonMap( + private static final Map>> TEST_ACCUMULATORS = Collections.singletonMap( TEST_ACCUMULATOR_NAME, SerializedValue.fromBytes(TEST_ACCUMULATOR_VALUE)); diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala index ce3231621c63d..c8529a9e07ac2 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala @@ -30,6 +30,7 @@ import org.apache.flink.runtime.instance.ActorGateway import org.apache.flink.runtime.jobgraph.JobStatus import org.apache.flink.runtime.messages.RequiresLeaderSessionID import org.apache.flink.runtime.messages.checkpoint.AbstractCheckpointMessage +import org.apache.flink.util.OptionalFailure object TestingJobManagerMessages { @@ -108,7 +109,7 @@ object TestingJobManagerMessages { * Reports updated accumulators back to the listener. */ case class UpdatedAccumulators(jobID: JobID, - userAccumulators: Map[String, Accumulator[_,_]]) + userAccumulators: Map[String, OptionalFailure[Accumulator[_,_]]]) /** Notifies the sender when the [[TestingJobManager]] has been elected as the leader * diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java index 282b192494336..3d90833cbd02f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java @@ -18,25 +18,24 @@ package org.apache.flink.test.accumulators; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.DoubleCounter; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; -import org.apache.flink.test.util.TestEnvironment; +import org.apache.flink.test.util.MiniClusterResource; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** @@ -45,84 +44,80 @@ * b) are not compatible with existing accumulator. */ public class AccumulatorErrorITCase extends TestLogger { - - private static LocalFlinkMiniCluster cluster; - - private static ExecutionEnvironment env; - - @BeforeClass - public static void startCluster() { + private static final String FAULTY_CLONE_ACCUMULATOR = "faulty-clone"; + private static final String FAULTY_MERGE_ACCUMULATOR = "faulty-merge"; + private static final String INCOMPATIBLE_ACCUMULATORS_NAME = "incompatible-accumulators"; + + @ClassRule + public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfiguration(), + 2, + 3)); + + public static Configuration getConfiguration() { Configuration config = new Configuration(); - config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); - config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3); config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 12L); - cluster = new LocalFlinkMiniCluster(config, false); - - cluster.start(); - - env = new TestEnvironment(cluster, 6, false); - } - - @AfterClass - public static void shutdownCluster() { - cluster.stop(); - cluster = null; + return config; } @Test public void testFaultyAccumulator() throws Exception { - + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.getConfig().disableSysoutLogging(); // Test Exception forwarding with faulty Accumulator implementation - DataSet input = env.generateSequence(0, 10000); + env.generateSequence(0, 10000) + .map(new FaultyAccumulatorUsingMapper()) + .output(new DiscardingOutputFormat<>()); - DataSet map = input.map(new FaultyAccumulatorUsingMapper()); - - map.output(new DiscardingOutputFormat()); - - try { - env.execute(); - fail("Should have failed."); - } catch (JobExecutionException e) { - Assert.assertTrue("Root cause should be:", - e.getCause() instanceof CustomException); - } + assertAccumulatorsShouldFail(env.execute()); } @Test public void testInvalidTypeAccumulator() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); env.getConfig().disableSysoutLogging(); // Test Exception forwarding with faulty Accumulator implementation - DataSet input = env.generateSequence(0, 10000); - - DataSet mappers = input.map(new IncompatibleAccumulatorTypesMapper()) - .map(new IncompatibleAccumulatorTypesMapper2()); - - mappers.output(new DiscardingOutputFormat()); + env.generateSequence(0, 10000) + .map(new IncompatibleAccumulatorTypesMapper()) + .map(new IncompatibleAccumulatorTypesMapper2()) + .output(new DiscardingOutputFormat<>()); try { env.execute(); fail("Should have failed."); } catch (JobExecutionException e) { - Assert.assertTrue("Root cause should be:", + assertTrue("Root cause should be:", e.getCause() instanceof Exception); - Assert.assertTrue("Root cause should be:", + assertTrue("Root cause should be:", e.getCause().getCause() instanceof UnsupportedOperationException); } } + @Test + public void testFaultyMergeAccumulator() throws Exception { + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().disableSysoutLogging(); + + // Test Exception forwarding with faulty Accumulator implementation + env.generateSequence(0, 10000) + .map(new FaultyMergeAccumulatorUsingMapper()) + .output(new DiscardingOutputFormat<>()); + + assertAccumulatorsShouldFail(env.execute()); + } + /* testFaultyAccumulator */ private static class FaultyAccumulatorUsingMapper extends RichMapFunction { - private static final long serialVersionUID = 42; @Override public void open(Configuration parameters) throws Exception { - getRuntimeContext().addAccumulator("test", new FaultyAccumulator()); + getRuntimeContext().addAccumulator(FAULTY_CLONE_ACCUMULATOR, new FaultyCloneAccumulator()); } @Override @@ -131,8 +126,7 @@ public Long map(Long value) throws Exception { } } - private static class FaultyAccumulator extends LongCounter { - + private static class FaultyCloneAccumulator extends LongCounter { private static final long serialVersionUID = 42; @Override @@ -141,19 +135,14 @@ public LongCounter clone() { } } - private static class CustomException extends RuntimeException { - private static final long serialVersionUID = 42; - } - /* testInvalidTypeAccumulator */ private static class IncompatibleAccumulatorTypesMapper extends RichMapFunction { - private static final long serialVersionUID = 42; @Override public void open(Configuration parameters) throws Exception { - getRuntimeContext().addAccumulator("test", new LongCounter()); + getRuntimeContext().addAccumulator(INCOMPATIBLE_ACCUMULATORS_NAME, new LongCounter()); } @Override @@ -163,12 +152,27 @@ public Long map(Long value) throws Exception { } private static class IncompatibleAccumulatorTypesMapper2 extends RichMapFunction { + private static final long serialVersionUID = 42; + + @Override + public void open(Configuration parameters) throws Exception { + getRuntimeContext().addAccumulator(INCOMPATIBLE_ACCUMULATORS_NAME, new DoubleCounter()); + } + + @Override + public Long map(Long value) throws Exception { + return -1L; + } + } + + /** */ + private static class FaultyMergeAccumulatorUsingMapper extends RichMapFunction { private static final long serialVersionUID = 42; @Override public void open(Configuration parameters) throws Exception { - getRuntimeContext().addAccumulator("test", new DoubleCounter()); + getRuntimeContext().addAccumulator(FAULTY_MERGE_ACCUMULATOR, new FaultyMergeAccumulator()); } @Override @@ -177,4 +181,31 @@ public Long map(Long value) throws Exception { } } + private static class FaultyMergeAccumulator extends LongCounter { + private static final long serialVersionUID = 42; + + @Override + public void merge(Accumulator other) { + throw new CustomException(); + } + + @Override + public LongCounter clone() { + return new FaultyMergeAccumulator(); + } + } + + private static class CustomException extends RuntimeException { + private static final long serialVersionUID = 42; + } + + private static void assertAccumulatorsShouldFail(JobExecutionResult result) { + try { + result.getAllAccumulatorResults(); + fail("Should have failed"); + } + catch (Exception ex) { + assertTrue(ExceptionUtils.findThrowable(ex, CustomException.class).isPresent()); + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java index b7f54facff4b4..9f87a5638f807 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorITCase.java @@ -76,6 +76,7 @@ protected void postSubmit() throws Exception { System.out.println(AccumulatorHelper.getResultsFormatted(res.getAllAccumulatorResults())); Assert.assertEquals(Integer.valueOf(3), res.getAccumulatorResult("num-lines")); + Assert.assertEquals(Integer.valueOf(3), res.getIntCounterResult("num-lines")); Assert.assertEquals(Double.valueOf(getParallelism()), res.getAccumulatorResult("open-close-counter")); diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index ff362dde50246..4a2219ae24b4a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -160,7 +160,7 @@ private static void submitJobAndVerifyResults(JobGraph jobGraph) throws Exceptio deadline, accumulators -> accumulators.size() == 1 && accumulators.containsKey(ACCUMULATOR_NAME) - && (int) accumulators.get(ACCUMULATOR_NAME) == NUM_ITERATIONS, + && (int) accumulators.get(ACCUMULATOR_NAME).getUnchecked() == NUM_ITERATIONS, TestingUtils.defaultScheduledExecutor() ).get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/LegacyAccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/LegacyAccumulatorLiveITCase.java index 6595b100c1581..b273ade2dd271 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/LegacyAccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/LegacyAccumulatorLiveITCase.java @@ -49,6 +49,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.util.Collector; +import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.TestLogger; import akka.actor.ActorRef; @@ -195,7 +196,7 @@ private static void verifyResults() { expectMsgClass(TIMEOUT, JobManagerMessages.JobSubmitSuccess.class); TestingJobManagerMessages.UpdatedAccumulators msg = (TestingJobManagerMessages.UpdatedAccumulators) receiveOne(TIMEOUT); - Map> userAccumulators = msg.userAccumulators(); + Map>> userAccumulators = msg.userAccumulators(); ExecutionAttemptID mapperTaskID = null; @@ -264,9 +265,9 @@ private static void verifyResults() { }}; } - private static boolean checkUserAccumulators(int expected, Map> accumulatorMap) { + private static boolean checkUserAccumulators(int expected, Map>> accumulatorMap) { LOG.info("checking user accumulators"); - return accumulatorMap.containsKey(ACCUMULATOR_NAME) && expected == ((IntCounter) accumulatorMap.get(ACCUMULATOR_NAME)).getLocalValue(); + return accumulatorMap.containsKey(ACCUMULATOR_NAME) && expected == ((IntCounter) accumulatorMap.get(ACCUMULATOR_NAME).getUnchecked()).getLocalValue(); } /** diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java index 91b5de8ca5e25..8f2aaa1b1fe02 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/SavepointMigrationTestBase.java @@ -32,6 +32,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.util.OptionalFailure; import org.apache.commons.io.FileUtils; import org.junit.BeforeClass; @@ -138,11 +139,11 @@ protected final void executeAndSavepoint( boolean done = false; while (DEADLINE.hasTimeLeft()) { Thread.sleep(100); - Map accumulators = client.getAccumulators(jobSubmissionResult.getJobID()); + Map> accumulators = client.getAccumulators(jobSubmissionResult.getJobID()); boolean allDone = true; for (Tuple2 acc : expectedAccumulators) { - Integer numFinished = (Integer) accumulators.get(acc.f0); + Integer numFinished = (Integer) accumulators.get(acc.f0).get(); if (numFinished == null) { allDone = false; break; @@ -211,16 +212,16 @@ protected final void restoreAndExecute( } Thread.sleep(100); - Map accumulators = client.getAccumulators(jobId); + Map> accumulators = client.getAccumulators(jobId); boolean allDone = true; for (Tuple2 acc : expectedAccumulators) { - Integer numFinished = (Integer) accumulators.get(acc.f0); + OptionalFailure numFinished = accumulators.get(acc.f0); if (numFinished == null) { allDone = false; break; } - if (!numFinished.equals(acc.f1)) { + if (!numFinished.get().equals(acc.f1)) { allDone = false; break; } From d721e20bfd866c6b5af642d263ad32e0ca59fa36 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 14 Mar 2018 15:38:48 +0100 Subject: [PATCH 215/268] [FLINK-8852] [sql-client] Add support for FLIP-6 in SQL Client This closes #5704. --- flink-clients/pom.xml | 12 + .../flink/client/cli/CliFrontendParser.java | 2 +- .../flink/client/program/ClusterClient.java | 10 +- .../conf/sql-client-defaults.yaml | 2 - flink-libraries/flink-sql-client/pom.xml | 9 + .../table/client/SqlClientException.java | 4 + .../flink/table/client/config/Deployment.java | 43 ++ .../local/ChangelogCollectStreamResult.java | 4 +- .../client/gateway/local/ChangelogResult.java | 4 +- .../gateway/local/CollectStreamResult.java | 13 +- .../client/gateway/local/DynamicResult.java | 9 +- .../gateway/local/ExecutionContext.java | 256 ++++++++---- .../client/gateway/local/LocalExecutor.java | 395 ++++++++++-------- .../MaterializedCollectStreamResult.java | 4 +- .../gateway/local/MaterializedResult.java | 4 +- .../client/gateway/local/ResultStore.java | 13 +- .../client/gateway/local/DependencyTest.java | 4 +- .../gateway/local/LocalExecutorITCase.java | 60 ++- .../resources/test-sql-client-defaults.yaml | 1 - .../resources/test-sql-client-factory.yaml | 1 - 20 files changed, 561 insertions(+), 289 deletions(-) diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index d175b8d031878..399d080ef67ad 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -147,6 +147,18 @@ under the License. + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index eb6826436a5c8..5a6c0ff1838cb 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -420,7 +420,7 @@ public static CommandLine parse(Options options, String[] args, boolean stopAtNo * @param optionsB options to merge, can be null if none * @return */ - static Options mergeOptions(@Nullable Options optionsA, @Nullable Options optionsB) { + public static Options mergeOptions(@Nullable Options optionsA, @Nullable Options optionsB) { final Options resultOptions = new Options(); if (optionsA != null) { for (Option option : optionsA.getOptions()) { diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index 79a5383109581..f50206d1492c4 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -460,7 +460,7 @@ public JobSubmissionResult run( public JobSubmissionResult run(FlinkPlan compiledPlan, List libraries, List classpaths, ClassLoader classLoader, SavepointRestoreSettings savepointSettings) throws ProgramInvocationException { - JobGraph job = getJobGraph(compiledPlan, libraries, classpaths, savepointSettings); + JobGraph job = getJobGraph(flinkConfig, compiledPlan, libraries, classpaths, savepointSettings); return submitJob(job, classLoader); } @@ -882,17 +882,17 @@ private static OptimizedPlan getOptimizedPlan(Optimizer compiler, JobWithJars pr return getOptimizedPlan(compiler, prog.getPlan(), parallelism); } - public JobGraph getJobGraph(PackagedProgram prog, FlinkPlan optPlan, SavepointRestoreSettings savepointSettings) throws ProgramInvocationException { - return getJobGraph(optPlan, prog.getAllLibraries(), prog.getClasspaths(), savepointSettings); + public static JobGraph getJobGraph(Configuration flinkConfig, PackagedProgram prog, FlinkPlan optPlan, SavepointRestoreSettings savepointSettings) throws ProgramInvocationException { + return getJobGraph(flinkConfig, optPlan, prog.getAllLibraries(), prog.getClasspaths(), savepointSettings); } - public JobGraph getJobGraph(FlinkPlan optPlan, List jarFiles, List classpaths, SavepointRestoreSettings savepointSettings) { + public static JobGraph getJobGraph(Configuration flinkConfig, FlinkPlan optPlan, List jarFiles, List classpaths, SavepointRestoreSettings savepointSettings) { JobGraph job; if (optPlan instanceof StreamingPlan) { job = ((StreamingPlan) optPlan).getJobGraph(); job.setSavepointRestoreSettings(savepointSettings); } else { - JobGraphGenerator gen = new JobGraphGenerator(this.flinkConfig); + JobGraphGenerator gen = new JobGraphGenerator(flinkConfig); job = gen.compileJobGraph((OptimizedPlan) optPlan); } diff --git a/flink-libraries/flink-sql-client/conf/sql-client-defaults.yaml b/flink-libraries/flink-sql-client/conf/sql-client-defaults.yaml index 35584222e22f9..4ec64d69e52d0 100644 --- a/flink-libraries/flink-sql-client/conf/sql-client-defaults.yaml +++ b/flink-libraries/flink-sql-client/conf/sql-client-defaults.yaml @@ -64,8 +64,6 @@ execution: # programs are submitted to. deployment: - # only the 'standalone' deployment is supported - type: standalone # general cluster communication timeout in ms response-timeout: 5000 # (optional) address from cluster to gateway diff --git a/flink-libraries/flink-sql-client/pom.xml b/flink-libraries/flink-sql-client/pom.xml index 71c0812c21226..a005f85037b1f 100644 --- a/flink-libraries/flink-sql-client/pom.xml +++ b/flink-libraries/flink-sql-client/pom.xml @@ -129,6 +129,15 @@ under the License. test + + org.apache.flink + + flink-clients_2.11 + ${project.version} + test-jar + test + + diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClientException.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClientException.java index bbd64b3a4959b..ae3b84c7be78c 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClientException.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/SqlClientException.java @@ -32,4 +32,8 @@ public SqlClientException(String message) { public SqlClientException(String message, Throwable e) { super(message, e); } + + public SqlClientException(Throwable e) { + super(e); + } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Deployment.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Deployment.java index a87f2e40287fd..0e809c47a49df 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Deployment.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/Deployment.java @@ -18,8 +18,16 @@ package org.apache.flink.table.client.config; +import org.apache.flink.client.cli.CliFrontendParser; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; + +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; @@ -58,6 +66,41 @@ public int getGatewayPort() { return Integer.parseInt(properties.getOrDefault(PropertyStrings.DEPLOYMENT_GATEWAY_PORT, Integer.toString(0))); } + /** + * Parses the given command line options from the deployment properties. Ignores properties + * that are not defined by options. + */ + public CommandLine getCommandLine(Options commandLineOptions) throws Exception { + final List args = new ArrayList<>(); + + properties.forEach((k, v) -> { + // only add supported options + if (commandLineOptions.hasOption(k)) { + final Option o = commandLineOptions.getOption(k); + final String argument = "--" + o.getLongOpt(); + // options without args + if (!o.hasArg()) { + final Boolean flag = Boolean.parseBoolean(v); + // add key only + if (flag) { + args.add(argument); + } + } + // add key and value + else if (!o.hasArgs()) { + args.add(argument); + args.add(v); + } + // options with multiple args are not supported yet + else { + throw new IllegalArgumentException("Option '" + o + "' is not supported yet."); + } + } + }); + + return CliFrontendParser.parse(commandLineOptions, args.toArray(new String[args.size()]), true); + } + public Map toProperties() { final Map copy = new HashMap<>(); properties.forEach((k, v) -> copy.put(PropertyStrings.DEPLOYMENT + "." + k, v)); diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ChangelogCollectStreamResult.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ChangelogCollectStreamResult.java index 4302481a8833b..237558454f2c0 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ChangelogCollectStreamResult.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ChangelogCollectStreamResult.java @@ -30,8 +30,10 @@ /** * Collects results and returns them as a changelog. + * + * @param cluster id to which this result belongs to */ -public class ChangelogCollectStreamResult extends CollectStreamResult implements ChangelogResult { +public class ChangelogCollectStreamResult extends CollectStreamResult implements ChangelogResult { private List> changeRecordBuffer; private static final int CHANGE_RECORD_BUFFER_SIZE = 5_000; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ChangelogResult.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ChangelogResult.java index d607bfafffae1..6d4f95acf6190 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ChangelogResult.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ChangelogResult.java @@ -26,8 +26,10 @@ /** * A result that is represented as a changelog consisting of insert and delete records. + * + * @param cluster id to which this result belongs to */ -public interface ChangelogResult extends DynamicResult { +public interface ChangelogResult extends DynamicResult { /** * Retrieves the available result records. diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectStreamResult.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectStreamResult.java index a6f52af0ff452..83f9ff2190017 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectStreamResult.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/CollectStreamResult.java @@ -37,8 +37,10 @@ /** * A result that works similarly to {@link DataStreamUtils#collect(DataStream)}. + * + * @param cluster id to which this result belongs to */ -public abstract class CollectStreamResult implements DynamicResult { +public abstract class CollectStreamResult implements DynamicResult { private final TypeInformation outputType; private final SocketStreamIterator> iterator; @@ -46,6 +48,7 @@ public abstract class CollectStreamResult implements DynamicResult { private final ResultRetrievalThread retrievalThread; private final JobMonitoringThread monitoringThread; private Runnable program; + private C clusterId; protected final Object resultLock; protected SqlExecutionException executionException; @@ -73,6 +76,14 @@ public CollectStreamResult(TypeInformation outputType, ExecutionConfig conf monitoringThread = new JobMonitoringThread(); } + @Override + public void setClusterId(C clusterId) { + if (this.clusterId != null) { + throw new IllegalStateException("Cluster id is already present."); + } + this.clusterId = clusterId; + } + @Override public TypeInformation getOutputType() { return outputType; diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/DynamicResult.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/DynamicResult.java index 432cc65d0445d..2042e1a4ccb0a 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/DynamicResult.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/DynamicResult.java @@ -26,8 +26,15 @@ * A result of a dynamic table program. * *

    Note: Make sure to call close() after the result is not needed anymore. + * + * @param cluster id to which this result belongs to */ -public interface DynamicResult { +public interface DynamicResult { + + /** + * Sets the cluster id of the cluster this result comes from. This method should only be called once. + */ + void setClusterId(C clusterId); /** * Returns whether this result is materialized such that snapshots can be taken or results diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index a013afcc0a424..81931e2641f0d 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -22,6 +22,11 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.cli.CliArgsException; +import org.apache.flink.client.cli.CustomCommandLine; +import org.apache.flink.client.cli.RunOptions; +import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.optimizer.DataStatistics; @@ -29,48 +34,56 @@ import org.apache.flink.optimizer.costs.DefaultCostEstimator; import org.apache.flink.optimizer.plan.FlinkPlan; import org.apache.flink.runtime.execution.librarycache.FlinkUserCodeClassLoaders; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.table.api.BatchQueryConfig; import org.apache.flink.table.api.QueryConfig; import org.apache.flink.table.api.StreamQueryConfig; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.client.config.Deployment; import org.apache.flink.table.client.config.Environment; import org.apache.flink.table.client.gateway.SessionContext; +import org.apache.flink.table.client.gateway.SqlExecutionException; import org.apache.flink.table.sources.TableSource; import org.apache.flink.table.sources.TableSourceFactoryService; +import org.apache.flink.util.FlinkException; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; import java.net.URL; +import java.util.HashMap; import java.util.List; +import java.util.Map; /** - * Context for executing table programs. It contains configured environments and environment - * specific logic such as plan translation. + * Context for executing table programs. This class caches everything that can be cached across + * multiple queries as long as the session context does not change. This must be thread-safe as + * it might be reused across different query submission. + * + * @param cluster id */ -public class ExecutionContext { +public class ExecutionContext { private final SessionContext sessionContext; private final Environment mergedEnv; - private final ExecutionEnvironment execEnv; - private final StreamExecutionEnvironment streamExecEnv; - private final TableEnvironment tableEnv; + private final List dependencies; private final ClassLoader classLoader; - private final QueryConfig queryConfig; + private final Map> tableSources; + private final Configuration flinkConfig; + private final CommandLine commandLine; + private final CustomCommandLine activeCommandLine; + private final RunOptions runOptions; + private final T clusterId; + private final ClusterSpecification clusterSpec; - public ExecutionContext(Environment defaultEnvironment, SessionContext sessionContext, List dependencies) { + public ExecutionContext(Environment defaultEnvironment, SessionContext sessionContext, List dependencies, + Configuration flinkConfig, Options commandLineOptions, List> availableCommandLines) { this.sessionContext = sessionContext; this.mergedEnv = Environment.merge(defaultEnvironment, sessionContext.getEnvironment()); - - // create environments - if (mergedEnv.getExecution().isStreamingExecution()) { - streamExecEnv = createStreamExecutionEnvironment(); - execEnv = null; - tableEnv = TableEnvironment.getTableEnvironment(streamExecEnv); - } else { - streamExecEnv = null; - execEnv = createExecutionEnvironment(); - tableEnv = TableEnvironment.getTableEnvironment(execEnv); - } + this.dependencies = dependencies; + this.flinkConfig = flinkConfig; // create class loader classLoader = FlinkUserCodeClassLoaders.parentFirst( @@ -78,90 +91,189 @@ public ExecutionContext(Environment defaultEnvironment, SessionContext sessionCo this.getClass().getClassLoader()); // create table sources + tableSources = new HashMap<>(); mergedEnv.getSources().forEach((name, source) -> { - TableSource tableSource = TableSourceFactoryService.findAndCreateTableSource(source, classLoader); - tableEnv.registerTableSource(name, tableSource); + final TableSource tableSource = TableSourceFactoryService.findAndCreateTableSource(source, classLoader); + tableSources.put(name, tableSource); }); - // create query config - queryConfig = createQueryConfig(); + // convert deployment options into command line options that describe a cluster + commandLine = createCommandLine(mergedEnv.getDeployment(), commandLineOptions); + activeCommandLine = findActiveCommandLine(availableCommandLines, commandLine); + runOptions = createRunOptions(commandLine); + clusterId = activeCommandLine.getClusterId(commandLine); + clusterSpec = createClusterSpecification(activeCommandLine, commandLine); } public SessionContext getSessionContext() { return sessionContext; } - public ExecutionEnvironment getExecutionEnvironment() { - return execEnv; + public ClassLoader getClassLoader() { + return classLoader; } - public StreamExecutionEnvironment getStreamExecutionEnvironment() { - return streamExecEnv; + public Environment getMergedEnvironment() { + return mergedEnv; } - public TableEnvironment getTableEnvironment() { - return tableEnv; + public ClusterSpecification getClusterSpec() { + return clusterSpec; } - public ClassLoader getClassLoader() { - return classLoader; + public T getClusterId() { + return clusterId; } - public Environment getMergedEnvironment() { - return mergedEnv; + public ClusterDescriptor createClusterDescriptor() throws Exception { + return activeCommandLine.createClusterDescriptor(commandLine); } - public QueryConfig getQueryConfig() { - return queryConfig; + public EnvironmentInstance createEnvironmentInstance() { + return new EnvironmentInstance(); } - public ExecutionConfig getExecutionConfig() { - if (streamExecEnv != null) { - return streamExecEnv.getConfig(); - } else { - return execEnv.getConfig(); + // -------------------------------------------------------------------------------------------- + + private static CommandLine createCommandLine(Deployment deployment, Options commandLineOptions) { + try { + return deployment.getCommandLine(commandLineOptions); + } catch (Exception e) { + throw new SqlExecutionException("Invalid deployment options.", e); } } - public FlinkPlan createPlan(String name, Configuration flinkConfig) { - if (streamExecEnv != null) { - final StreamGraph graph = streamExecEnv.getStreamGraph(); - graph.setJobName(name); - return graph; - } else { - final int parallelism = execEnv.getParallelism(); - final Plan unoptimizedPlan = execEnv.createProgramPlan(); - unoptimizedPlan.setJobName(name); - final Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig); - return ClusterClient.getOptimizedPlan(compiler, unoptimizedPlan, parallelism); + @SuppressWarnings("unchecked") + private static CustomCommandLine findActiveCommandLine(List> availableCommandLines, CommandLine commandLine) { + for (CustomCommandLine cli : availableCommandLines) { + if (cli.isActive(commandLine)) { + return (CustomCommandLine) cli; + } } + throw new SqlExecutionException("Could not find a matching deployment."); } - // -------------------------------------------------------------------------------------------- - - private ExecutionEnvironment createExecutionEnvironment() { - final ExecutionEnvironment execEnv = ExecutionEnvironment.getExecutionEnvironment(); - execEnv.setParallelism(mergedEnv.getExecution().getParallelism()); - return execEnv; + private static RunOptions createRunOptions(CommandLine commandLine) { + try { + return new RunOptions(commandLine); + } catch (CliArgsException e) { + throw new SqlExecutionException("Invalid deployment run options.", e); + } } - private StreamExecutionEnvironment createStreamExecutionEnvironment() { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(mergedEnv.getExecution().getParallelism()); - env.setMaxParallelism(mergedEnv.getExecution().getMaxParallelism()); - env.setStreamTimeCharacteristic(mergedEnv.getExecution().getTimeCharacteristic()); - return env; + private static ClusterSpecification createClusterSpecification(CustomCommandLine activeCommandLine, CommandLine commandLine) { + try { + return activeCommandLine.getClusterSpecification(commandLine); + } catch (FlinkException e) { + throw new SqlExecutionException("Could not create cluster specification for the given deployment.", e); + } } - private QueryConfig createQueryConfig() { - if (streamExecEnv != null) { - final StreamQueryConfig config = new StreamQueryConfig(); - final long minRetention = mergedEnv.getExecution().getMinStateRetention(); - final long maxRetention = mergedEnv.getExecution().getMaxStateRetention(); - config.withIdleStateRetentionTime(Time.milliseconds(minRetention), Time.milliseconds(maxRetention)); - return config; - } else { - return new BatchQueryConfig(); + // -------------------------------------------------------------------------------------------- + + /** + * {@link ExecutionEnvironment} and {@link StreamExecutionEnvironment} cannot be reused + * across multiple queries because they are stateful. This class abstracts execution + * environments and table environments. + */ + public class EnvironmentInstance { + + private final QueryConfig queryConfig; + private final ExecutionEnvironment execEnv; + private final StreamExecutionEnvironment streamExecEnv; + private final TableEnvironment tableEnv; + + private EnvironmentInstance() { + // create environments + if (mergedEnv.getExecution().isStreamingExecution()) { + streamExecEnv = createStreamExecutionEnvironment(); + execEnv = null; + tableEnv = TableEnvironment.getTableEnvironment(streamExecEnv); + } else { + streamExecEnv = null; + execEnv = createExecutionEnvironment(); + tableEnv = TableEnvironment.getTableEnvironment(execEnv); + } + + // create query config + queryConfig = createQueryConfig(); + + // register table sources + tableSources.forEach(tableEnv::registerTableSource); + } + + public QueryConfig getQueryConfig() { + return queryConfig; + } + + public ExecutionEnvironment getExecutionEnvironment() { + return execEnv; + } + + public StreamExecutionEnvironment getStreamExecutionEnvironment() { + return streamExecEnv; + } + + public TableEnvironment getTableEnvironment() { + return tableEnv; + } + + public ExecutionConfig getExecutionConfig() { + if (streamExecEnv != null) { + return streamExecEnv.getConfig(); + } else { + return execEnv.getConfig(); + } + } + + public JobGraph createJobGraph(String name) { + final FlinkPlan plan = createPlan(name, flinkConfig); + return ClusterClient.getJobGraph( + flinkConfig, + plan, + dependencies, + runOptions.getClasspaths(), + runOptions.getSavepointRestoreSettings()); + } + + private FlinkPlan createPlan(String name, Configuration flinkConfig) { + if (streamExecEnv != null) { + final StreamGraph graph = streamExecEnv.getStreamGraph(); + graph.setJobName(name); + return graph; + } else { + final int parallelism = execEnv.getParallelism(); + final Plan unoptimizedPlan = execEnv.createProgramPlan(); + unoptimizedPlan.setJobName(name); + final Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), flinkConfig); + return ClusterClient.getOptimizedPlan(compiler, unoptimizedPlan, parallelism); + } + } + + private ExecutionEnvironment createExecutionEnvironment() { + final ExecutionEnvironment execEnv = ExecutionEnvironment.getExecutionEnvironment(); + execEnv.setParallelism(mergedEnv.getExecution().getParallelism()); + return execEnv; + } + + private StreamExecutionEnvironment createStreamExecutionEnvironment() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(mergedEnv.getExecution().getParallelism()); + env.setMaxParallelism(mergedEnv.getExecution().getMaxParallelism()); + env.setStreamTimeCharacteristic(mergedEnv.getExecution().getTimeCharacteristic()); + return env; + } + + private QueryConfig createQueryConfig() { + if (streamExecEnv != null) { + final StreamQueryConfig config = new StreamQueryConfig(); + final long minRetention = mergedEnv.getExecution().getMinStateRetention(); + final long maxRetention = mergedEnv.getExecution().getMaxStateRetention(); + config.withIdleStateRetentionTime(Time.milliseconds(minRetention), Time.milliseconds(maxRetention)); + return config; + } else { + return new BatchQueryConfig(); + } } } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index fa6c9d2fd26cd..30fa3c0998a9d 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -21,35 +21,32 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.cli.CliFrontend; +import org.apache.flink.client.cli.CliFrontendParser; +import org.apache.flink.client.cli.CustomCommandLine; import org.apache.flink.client.deployment.ClusterDescriptor; -import org.apache.flink.client.deployment.ClusterRetrieveException; -import org.apache.flink.client.deployment.StandaloneClusterDescriptor; -import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.JobWithJars; -import org.apache.flink.client.program.ProgramInvocationException; -import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; -import org.apache.flink.optimizer.plan.FlinkPlan; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.client.SqlClientException; -import org.apache.flink.table.client.config.Deployment; import org.apache.flink.table.client.config.Environment; import org.apache.flink.table.client.gateway.Executor; import org.apache.flink.table.client.gateway.ResultDescriptor; import org.apache.flink.table.client.gateway.SessionContext; import org.apache.flink.table.client.gateway.SqlExecutionException; import org.apache.flink.table.client.gateway.TypedResult; -import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; import org.apache.flink.util.StringUtils; +import org.apache.commons.cli.Options; + import java.io.File; import java.io.IOException; import java.net.MalformedURLException; @@ -69,16 +66,23 @@ public class LocalExecutor implements Executor { private static final String DEFAULT_ENV_FILE = "sql-client-defaults.yaml"; + // deployment + private final Environment defaultEnvironment; private final List dependencies; private final Configuration flinkConfig; + private final List> commandLines; + private final Options commandLineOptions; + + // result maintenance + private final ResultStore resultStore; /** * Cached execution context for unmodified sessions. Do not access this variable directly * but through {@link LocalExecutor#getOrCreateExecutionContext}. */ - private ExecutionContext executionContext; + private ExecutionContext executionContext; /** * Creates a local executor for submitting table programs and retrieving results. @@ -92,6 +96,18 @@ public LocalExecutor(URL defaultEnv, List jars, List libraries) { // load the global configuration this.flinkConfig = GlobalConfiguration.loadConfiguration(flinkConfigDir); + + // initialize default file system + try { + FileSystem.initialize(this.flinkConfig); + } catch (IOException e) { + throw new SqlClientException( + "Error while setting the default filesystem scheme from configuration.", e); + } + + // load command lines for deployment + this.commandLines = CliFrontend.loadCustomCommandLines(flinkConfig, flinkConfigDir); + this.commandLineOptions = collectCommandLineOptions(commandLines); } catch (Exception e) { throw new SqlClientException("Could not load Flink configuration.", e); } @@ -107,7 +123,7 @@ public LocalExecutor(URL defaultEnv, List jars, List libraries) { try { defaultEnv = Path.fromLocalFile(file).toUri().toURL(); } catch (MalformedURLException e) { - throw new RuntimeException(e); + throw new SqlClientException(e); } } else { System.out.println("not found."); @@ -127,38 +143,7 @@ public LocalExecutor(URL defaultEnv, List jars, List libraries) { } // discover dependencies - dependencies = new ArrayList<>(); - try { - // find jar files - for (URL url : jars) { - JobWithJars.checkJarFile(url); - dependencies.add(url); - } - - // find jar files in library directories - for (URL libUrl : libraries) { - final File dir = new File(libUrl.toURI()); - if (!dir.isDirectory()) { - throw new SqlClientException("Directory expected: " + dir); - } else if (!dir.canRead()) { - throw new SqlClientException("Directory cannot be read: " + dir); - } - final File[] files = dir.listFiles(); - if (files == null) { - throw new SqlClientException("Directory cannot be read: " + dir); - } - for (File f : files) { - // only consider jars - if (f.isFile() && f.getAbsolutePath().toLowerCase().endsWith(".jar")) { - final URL url = f.toURI().toURL(); - JobWithJars.checkJarFile(url); - dependencies.add(url); - } - } - } - } catch (Exception e) { - throw new SqlClientException("Could not load all required JAR files.", e); - } + dependencies = discoverDependencies(jars, libraries); // prepare result store resultStore = new ResultStore(flinkConfig); @@ -167,10 +152,12 @@ public LocalExecutor(URL defaultEnv, List jars, List libraries) { /** * Constructor for testing purposes. */ - public LocalExecutor(Environment defaultEnvironment, List dependencies, Configuration flinkConfig) { + public LocalExecutor(Environment defaultEnvironment, List dependencies, Configuration flinkConfig, CustomCommandLine commandLine) { this.defaultEnvironment = defaultEnvironment; this.dependencies = dependencies; this.flinkConfig = flinkConfig; + this.commandLines = Collections.singletonList(commandLine); + this.commandLineOptions = collectCommandLineOptions(commandLines); // prepare result store resultStore = new ResultStore(flinkConfig); @@ -183,7 +170,8 @@ public void start() { @Override public Map getSessionProperties(SessionContext session) throws SqlExecutionException { - final Environment env = getOrCreateExecutionContext(session).getMergedEnvironment(); + final Environment env = getOrCreateExecutionContext(session) + .getMergedEnvironment(); final Map properties = new HashMap<>(); properties.putAll(env.getExecution().toProperties()); properties.putAll(env.getDeployment().toProperties()); @@ -192,13 +180,17 @@ public Map getSessionProperties(SessionContext session) throws S @Override public List listTables(SessionContext session) throws SqlExecutionException { - final TableEnvironment tableEnv = getOrCreateExecutionContext(session).getTableEnvironment(); + final TableEnvironment tableEnv = getOrCreateExecutionContext(session) + .createEnvironmentInstance() + .getTableEnvironment(); return Arrays.asList(tableEnv.listTables()); } @Override public TableSchema getTableSchema(SessionContext session, String name) throws SqlExecutionException { - final TableEnvironment tableEnv = getOrCreateExecutionContext(session).getTableEnvironment(); + final TableEnvironment tableEnv = getOrCreateExecutionContext(session) + .createEnvironmentInstance() + .getTableEnvironment(); try { return tableEnv.scan(name).getSchema(); } catch (Throwable t) { @@ -209,12 +201,14 @@ public TableSchema getTableSchema(SessionContext session, String name) throws Sq @Override public String explainStatement(SessionContext session, String statement) throws SqlExecutionException { - final ExecutionContext context = getOrCreateExecutionContext(session); + final TableEnvironment tableEnv = getOrCreateExecutionContext(session) + .createEnvironmentInstance() + .getTableEnvironment(); // translate try { - final Table table = createTable(context, statement); - return context.getTableEnvironment().explain(table); + final Table table = createTable(tableEnv, statement); + return tableEnv.explain(table); } catch (Throwable t) { // catch everything such that the query does not crash the executor throw new SqlExecutionException("Invalid SQL statement.", t); @@ -223,60 +217,8 @@ public String explainStatement(SessionContext session, String statement) throws @Override public ResultDescriptor executeQuery(SessionContext session, String query) throws SqlExecutionException { - final ExecutionContext context = getOrCreateExecutionContext(session); - final Environment mergedEnv = context.getMergedEnvironment(); - - // create table here to fail quickly for wrong queries - final Table table = createTable(context, query); - final TableSchema resultSchema = table.getSchema().withoutTimeAttributes(); - - // deployment - final ClusterClient clusterClient = createDeployment(mergedEnv.getDeployment()); - - // initialize result - final DynamicResult result = resultStore.createResult( - mergedEnv, - resultSchema, - context.getExecutionConfig()); - - // create job graph with jars - final JobGraph jobGraph; - try { - jobGraph = createJobGraph(context, context.getSessionContext().getName() + ": " + query, table, - result.getTableSink(), - clusterClient); - } catch (Throwable t) { - // the result needs to be closed as long as - // it not stored in the result store - result.close(); - throw t; - } - - // store the result with a unique id (the job id for now) - final String resultId = jobGraph.getJobID().toString(); - resultStore.storeResult(resultId, result); - - // create execution - final Runnable program = () -> { - // we need to submit the job attached for now - // otherwise it is not possible to retrieve the reason why an execution failed - try { - clusterClient.run(jobGraph, context.getClassLoader()); - } catch (ProgramInvocationException e) { - throw new SqlExecutionException("Could not execute table program.", e); - } finally { - try { - clusterClient.shutdown(); - } catch (Exception e) { - // ignore - } - } - }; - - // start result retrieval - result.startRetrieval(program); - - return new ResultDescriptor(resultId, resultSchema, result.isMaterialized()); + final ExecutionContext context = getOrCreateExecutionContext(session); + return executeQueryInternal(context, query); } @Override @@ -289,7 +231,7 @@ public TypedResult>> retrieveResultChanges(SessionCont if (result.isMaterialized()) { throw new SqlExecutionException("Invalid result retrieval mode."); } - return ((ChangelogResult) result).retrieveChanges(); + return ((ChangelogResult) result).retrieveChanges(); } @Override @@ -301,7 +243,7 @@ public TypedResult snapshotResult(SessionContext session, String result if (!result.isMaterialized()) { throw new SqlExecutionException("Invalid result retrieval mode."); } - return ((MaterializedResult) result).snapshot(pageSize); + return ((MaterializedResult) result).snapshot(pageSize); } @Override @@ -313,34 +255,13 @@ public List retrieveResultPage(String resultId, int page) throws SqlExecuti if (!result.isMaterialized()) { throw new SqlExecutionException("Invalid result retrieval mode."); } - return ((MaterializedResult) result).retrievePage(page); + return ((MaterializedResult) result).retrievePage(page); } @Override public void cancelQuery(SessionContext session, String resultId) throws SqlExecutionException { - final DynamicResult result = resultStore.getResult(resultId); - if (result == null) { - throw new SqlExecutionException("Could not find a result with result identifier '" + resultId + "'."); - } - - // stop retrieval and remove the result - result.close(); - resultStore.removeResult(resultId); - - // stop Flink job - final Environment mergedEnv = getOrCreateExecutionContext(session).getMergedEnvironment(); - final ClusterClient clusterClient = createDeployment(mergedEnv.getDeployment()); - try { - clusterClient.cancel(new JobID(StringUtils.hexStringToByte(resultId))); - } catch (Throwable t) { - // the job might has finished earlier - } finally { - try { - clusterClient.shutdown(); - } catch (Throwable t) { - // ignore - } - } + final ExecutionContext context = getOrCreateExecutionContext(session); + cancelQueryInternal(context, resultId); } @Override @@ -356,72 +277,105 @@ public void stop(SessionContext session) { // -------------------------------------------------------------------------------------------- - private Table createTable(ExecutionContext context, String query) { - // parse and validate query - try { - return context.getTableEnvironment().sqlQuery(query); - } catch (Throwable t) { - // catch everything such that the query does not crash the executor - throw new SqlExecutionException("Invalid SQL statement.", t); + private void cancelQueryInternal(ExecutionContext context, String resultId) { + final DynamicResult result = resultStore.getResult(resultId); + if (result == null) { + throw new SqlExecutionException("Could not find a result with result identifier '" + resultId + "'."); + } + + // stop retrieval and remove the result + result.close(); + resultStore.removeResult(resultId); + + // stop Flink job + try (final ClusterDescriptor clusterDescriptor = context.createClusterDescriptor()) { + ClusterClient clusterClient = null; + try { + // retrieve existing cluster + clusterClient = clusterDescriptor.retrieve(context.getClusterId()); + try { + clusterClient.cancel(new JobID(StringUtils.hexStringToByte(resultId))); + } catch (Throwable t) { + // the job might has finished earlier + } + } catch (Exception e) { + throw new SqlExecutionException("Could not retrieve or create a cluster.", e); + } finally { + try { + if (clusterClient != null) { + clusterClient.shutdown(); + } + } catch (Exception e) { + // ignore + } + } + } catch (SqlExecutionException e) { + throw e; + } catch (Exception e) { + throw new SqlExecutionException("Could not locate a cluster.", e); } } - private JobGraph createJobGraph(ExecutionContext context, String name, Table table, - TableSink sink, ClusterClient clusterClient) { + private ResultDescriptor executeQueryInternal(ExecutionContext context, String query) { + final ExecutionContext.EnvironmentInstance envInst = context.createEnvironmentInstance(); - // translate + // create table + final Table table = createTable(envInst.getTableEnvironment(), query); + + // initialize result + final DynamicResult result = resultStore.createResult( + context.getMergedEnvironment(), + table.getSchema().withoutTimeAttributes(), + envInst.getExecutionConfig()); + + // create job graph with dependencies + final String jobName = context.getSessionContext().getName() + ": " + query; + final JobGraph jobGraph; try { - table.writeToSink(sink, context.getQueryConfig()); + table.writeToSink(result.getTableSink(), envInst.getQueryConfig()); + jobGraph = envInst.createJobGraph(jobName); } catch (Throwable t) { + // the result needs to be closed as long as + // it not stored in the result store + result.close(); // catch everything such that the query does not crash the executor throw new SqlExecutionException("Invalid SQL statement.", t); } - // extract plan - final FlinkPlan plan = context.createPlan(name, clusterClient.getFlinkConfiguration()); - - // create job graph - return clusterClient.getJobGraph( - plan, - dependencies, - Collections.emptyList(), - SavepointRestoreSettings.none()); - } - - private ClusterClient createDeployment(Deployment deploy) { + // store the result with a unique id (the job id for now) + final String resultId = jobGraph.getJobID().toString(); + resultStore.storeResult(resultId, result); - // change some configuration options for being more responsive - flinkConfig.setString(AkkaOptions.LOOKUP_TIMEOUT, deploy.getResponseTimeout() + " ms"); - flinkConfig.setString(AkkaOptions.CLIENT_TIMEOUT, deploy.getResponseTimeout() + " ms"); + // create execution + final Runnable program = () -> deployJob(context, jobGraph, result); - // get cluster client - final ClusterClient clusterClient; - if (deploy.isStandaloneDeployment()) { - clusterClient = createStandaloneClusterClient(flinkConfig); - clusterClient.setPrintStatusDuringExecution(false); - } else { - throw new SqlExecutionException("Unsupported deployment."); - } + // start result retrieval + result.startRetrieval(program); - return clusterClient; + return new ResultDescriptor( + resultId, + table.getSchema().withoutTimeAttributes(), + result.isMaterialized()); } - private ClusterClient createStandaloneClusterClient(Configuration configuration) { - final ClusterDescriptor descriptor = new StandaloneClusterDescriptor(configuration); + private Table createTable(TableEnvironment tableEnv, String query) { + // parse and validate query try { - return descriptor.retrieve(StandaloneClusterId.getInstance()); - } catch (ClusterRetrieveException e) { - throw new SqlExecutionException("Could not retrievePage standalone cluster.", e); + return tableEnv.sqlQuery(query); + } catch (Throwable t) { + // catch everything such that the query does not crash the executor + throw new SqlExecutionException("Invalid SQL statement.", t); } } /** * Creates or reuses the execution context. */ - private synchronized ExecutionContext getOrCreateExecutionContext(SessionContext session) throws SqlExecutionException { + private synchronized ExecutionContext getOrCreateExecutionContext(SessionContext session) throws SqlExecutionException { if (executionContext == null || !executionContext.getSessionContext().equals(session)) { try { - executionContext = new ExecutionContext(defaultEnvironment, session, dependencies); + executionContext = new ExecutionContext<>(defaultEnvironment, session, dependencies, + flinkConfig, commandLineOptions, commandLines); } catch (Throwable t) { // catch everything such that a configuration does not crash the executor throw new SqlExecutionException("Could not create execution context.", t); @@ -429,4 +383,101 @@ private synchronized ExecutionContext getOrCreateExecutionContext(SessionContext } return executionContext; } + + /** + * Deploys a job. Depending on the deployment create a new job cluster. It saves cluster id in + * the result and blocks until job completion. + */ + private void deployJob(ExecutionContext context, JobGraph jobGraph, DynamicResult result) { + // create or retrieve cluster and deploy job + try (final ClusterDescriptor clusterDescriptor = context.createClusterDescriptor()) { + ClusterClient clusterClient = null; + try { + // new cluster + if (context.getClusterId() == null) { + // deploy job cluster with job attached + clusterClient = clusterDescriptor.deployJobCluster(context.getClusterSpec(), jobGraph, false); + // save the new cluster id + result.setClusterId(clusterClient.getClusterId()); + // we need to hard cast for now + ((RestClusterClient) clusterClient) + .requestJobResult(jobGraph.getJobID()) + .get() + .toJobExecutionResult(context.getClassLoader()); // throws exception if job fails + } + // reuse existing cluster + else { + // retrieve existing cluster + clusterClient = clusterDescriptor.retrieve(context.getClusterId()); + // save the cluster id + result.setClusterId(clusterClient.getClusterId()); + // submit the job + clusterClient.setDetached(false); + clusterClient.submitJob(jobGraph, context.getClassLoader()); // throws exception if job fails + } + } catch (Exception e) { + throw new SqlExecutionException("Could not retrieve or create a cluster.", e); + } finally { + try { + if (clusterClient != null) { + clusterClient.shutdown(); + } + } catch (Exception e) { + // ignore + } + } + } catch (SqlExecutionException e) { + throw e; + } catch (Exception e) { + throw new SqlExecutionException("Could not locate a cluster.", e); + } + } + + // -------------------------------------------------------------------------------------------- + + private static List discoverDependencies(List jars, List libraries) { + final List dependencies = new ArrayList<>(); + try { + // find jar files + for (URL url : jars) { + JobWithJars.checkJarFile(url); + dependencies.add(url); + } + + // find jar files in library directories + for (URL libUrl : libraries) { + final File dir = new File(libUrl.toURI()); + if (!dir.isDirectory()) { + throw new SqlClientException("Directory expected: " + dir); + } else if (!dir.canRead()) { + throw new SqlClientException("Directory cannot be read: " + dir); + } + final File[] files = dir.listFiles(); + if (files == null) { + throw new SqlClientException("Directory cannot be read: " + dir); + } + for (File f : files) { + // only consider jars + if (f.isFile() && f.getAbsolutePath().toLowerCase().endsWith(".jar")) { + final URL url = f.toURI().toURL(); + JobWithJars.checkJarFile(url); + dependencies.add(url); + } + } + } + } catch (Exception e) { + throw new SqlClientException("Could not load all required JAR files.", e); + } + return dependencies; + } + + private static Options collectCommandLineOptions(List> commandLines) { + final Options customOptions = new Options(); + for (CustomCommandLine customCommandLine : commandLines) { + customCommandLine.addRunOptions(customOptions); + } + return CliFrontendParser.mergeOptions( + CliFrontendParser.getRunCommandOptions(), + customOptions); + } } diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedCollectStreamResult.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedCollectStreamResult.java index 7935da63e0bf8..0ce270ee13f20 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedCollectStreamResult.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedCollectStreamResult.java @@ -33,8 +33,10 @@ /** * Collects results and returns them as table snapshots. + * + * @param cluster id to which this result belongs to */ -public class MaterializedCollectStreamResult extends CollectStreamResult implements MaterializedResult { +public class MaterializedCollectStreamResult extends CollectStreamResult implements MaterializedResult { private final List materializedTable; private final Map> rowPositions; // positions of rows in table for faster access diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedResult.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedResult.java index 9306e7863efa9..858af4d931c13 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedResult.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/MaterializedResult.java @@ -25,8 +25,10 @@ /** * A result that is materialized and can be viewed by navigating through a snapshot. + * + * @param cluster id to which this result belongs to */ -public interface MaterializedResult extends DynamicResult { +public interface MaterializedResult extends DynamicResult { /** * Takes a snapshot of the current table and returns the number of pages for navigating diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java index 19a440e22246b..7e17ee7d186ba 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ResultStore.java @@ -46,7 +46,7 @@ public class ResultStore { private Configuration flinkConfig; - private Map results; + private Map> results; public ResultStore(Configuration flinkConfig) { this.flinkConfig = flinkConfig; @@ -57,7 +57,7 @@ public ResultStore(Configuration flinkConfig) { /** * Creates a result. Might start threads or opens sockets so every created result must be closed. */ - public DynamicResult createResult(Environment env, TableSchema schema, ExecutionConfig config) { + public DynamicResult createResult(Environment env, TableSchema schema, ExecutionConfig config) { if (!env.getExecution().isStreamingExecution()) { throw new SqlExecutionException("Emission is only supported in streaming environments yet."); } @@ -68,9 +68,9 @@ public DynamicResult createResult(Environment env, TableSchema schema, Execution final int gatewayPort = getGatewayPort(env.getDeployment()); if (env.getExecution().isChangelogMode()) { - return new ChangelogCollectStreamResult(outputType, config, gatewayAddress, gatewayPort); + return new ChangelogCollectStreamResult<>(outputType, config, gatewayAddress, gatewayPort); } else { - return new MaterializedCollectStreamResult(outputType, config, gatewayAddress, gatewayPort); + return new MaterializedCollectStreamResult<>(outputType, config, gatewayAddress, gatewayPort); } } @@ -78,8 +78,9 @@ public void storeResult(String resultId, DynamicResult result) { results.put(resultId, result); } - public DynamicResult getResult(String resultId) { - return results.get(resultId); + @SuppressWarnings("unchecked") + public DynamicResult getResult(String resultId) { + return (DynamicResult) results.get(resultId); } public void removeResult(String resultId) { diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java index 40a1c2cf137c2..bbb2024c649e4 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java @@ -18,6 +18,7 @@ package org.apache.flink.table.client.gateway.local; +import org.apache.flink.client.cli.Flip6DefaultCLI; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; @@ -57,7 +58,8 @@ public void testTableSourceFactoryDiscovery() throws Exception { final LocalExecutor executor = new LocalExecutor( env, Collections.singletonList(dependency), - new Configuration()); + new Configuration(), + new Flip6DefaultCLI(new Configuration())); final SessionContext session = new SessionContext("test-session", new Environment()); diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java index 45369784f563c..378d475107a99 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/LocalExecutorITCase.java @@ -22,9 +22,12 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.cli.util.DummyCustomCommandLine; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.minicluster.StandaloneMiniCluster; +import org.apache.flink.configuration.WebOptions; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.client.config.Environment; import org.apache.flink.table.client.gateway.Executor; @@ -32,12 +35,13 @@ import org.apache.flink.table.client.gateway.SessionContext; import org.apache.flink.table.client.gateway.TypedResult; import org.apache.flink.table.client.gateway.utils.EnvironmentFileUtil; +import org.apache.flink.test.util.MiniClusterResource; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; import org.apache.flink.util.TestLogger; -import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import java.net.URL; @@ -62,25 +66,36 @@ public class LocalExecutorITCase extends TestLogger { private static final String DEFAULTS_ENVIRONMENT_FILE = "test-sql-client-defaults.yaml"; - private static StandaloneMiniCluster cluster; + private static final int NUM_TMS = 2; + private static final int NUM_SLOTS_PER_TM = 2; - @BeforeClass - public static void before() throws Exception { - final Configuration config = new Configuration(); - config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 1); + @ClassRule + public static final MiniClusterResource MINI_CLUSTER_RESOURCE = new MiniClusterResource( + new MiniClusterResource.MiniClusterResourceConfiguration( + getConfig(), + NUM_TMS, + NUM_SLOTS_PER_TM), + true); + + private static ClusterClient clusterClient; - cluster = new StandaloneMiniCluster(config); + @BeforeClass + public static void setup() { + clusterClient = MINI_CLUSTER_RESOURCE.getClusterClient(); } - @AfterClass - public static void after() throws Exception { - cluster.close(); - cluster = null; + private static Configuration getConfig() { + Configuration config = new Configuration(); + config.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, 4L); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TMS); + config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, NUM_SLOTS_PER_TM); + config.setBoolean(WebOptions.SUBMIT_ENABLE, false); + return config; } @Test public void testListTables() throws Exception { - final Executor executor = createDefaultExecutor(); + final Executor executor = createDefaultExecutor(clusterClient); final SessionContext session = new SessionContext("test-session", new Environment()); final List actualTables = executor.listTables(session); @@ -91,7 +106,7 @@ public void testListTables() throws Exception { @Test public void testGetSessionProperties() throws Exception { - final Executor executor = createDefaultExecutor(); + final Executor executor = createDefaultExecutor(clusterClient); final SessionContext session = new SessionContext("test-session", new Environment()); // modify defaults @@ -107,7 +122,6 @@ public void testGetSessionProperties() throws Exception { expectedProperties.put("execution.max-idle-state-retention", "0"); expectedProperties.put("execution.min-idle-state-retention", "0"); expectedProperties.put("execution.result-mode", "table"); - expectedProperties.put("deployment.type", "standalone"); expectedProperties.put("deployment.response-timeout", "5000"); assertEquals(expectedProperties, actualProperties); @@ -115,7 +129,7 @@ public void testGetSessionProperties() throws Exception { @Test public void testTableSchema() throws Exception { - final Executor executor = createDefaultExecutor(); + final Executor executor = createDefaultExecutor(clusterClient); final SessionContext session = new SessionContext("test-session", new Environment()); final TableSchema actualTableSchema = executor.getTableSchema(session, "TableNumber2"); @@ -136,7 +150,7 @@ public void testQueryExecutionChangelog() throws Exception { replaceVars.put("$VAR_1", "/"); replaceVars.put("$VAR_2", "changelog"); - final Executor executor = createModifiedExecutor(replaceVars); + final Executor executor = createModifiedExecutor(clusterClient, replaceVars); final SessionContext session = new SessionContext("test-session", new Environment()); try { @@ -183,7 +197,7 @@ public void testQueryExecutionTable() throws Exception { replaceVars.put("$VAR_1", "/"); replaceVars.put("$VAR_2", "table"); - final Executor executor = createModifiedExecutor(replaceVars); + final Executor executor = createModifiedExecutor(clusterClient, replaceVars); final SessionContext session = new SessionContext("test-session", new Environment()); try { @@ -223,17 +237,19 @@ public void testQueryExecutionTable() throws Exception { } } - private LocalExecutor createDefaultExecutor() throws Exception { + private LocalExecutor createDefaultExecutor(ClusterClient clusterClient) throws Exception { return new LocalExecutor( EnvironmentFileUtil.parseUnmodified(DEFAULTS_ENVIRONMENT_FILE), Collections.emptyList(), - cluster.getConfiguration()); + clusterClient.getFlinkConfiguration(), + new DummyCustomCommandLine(clusterClient)); } - private LocalExecutor createModifiedExecutor(Map replaceVars) throws Exception { + private LocalExecutor createModifiedExecutor(ClusterClient clusterClient, Map replaceVars) throws Exception { return new LocalExecutor( EnvironmentFileUtil.parseModified(DEFAULTS_ENVIRONMENT_FILE, replaceVars), Collections.emptyList(), - cluster.getConfiguration()); + clusterClient.getFlinkConfiguration(), + new DummyCustomCommandLine(clusterClient)); } } diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml index 5a598f15ab42b..8f11d2332532f 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-defaults.yaml @@ -71,7 +71,6 @@ execution: result-mode: "$VAR_2" deployment: - type: standalone response-timeout: 5000 diff --git a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml index daa1fd167b5f2..d0caf842c8950 100644 --- a/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml +++ b/flink-libraries/flink-sql-client/src/test/resources/test-sql-client-factory.yaml @@ -46,7 +46,6 @@ execution: parallelism: 1 deployment: - type: standalone response-timeout: 5000 From ba73f707c3f07209fcdb0318d6e1424e2eb48da8 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Wed, 21 Mar 2018 16:21:46 +0100 Subject: [PATCH 216/268] [FLINK-8972] [e2eTests] Add DataSetAllroundTestProgram and e2e test script. --- .../flink-dataset-allround-test/pom.xml | 104 +++++++ .../tests/DataSetAllroundTestProgram.java | 285 ++++++++++++++++++ flink-end-to-end-tests/pom.xml | 1 + flink-end-to-end-tests/run-nightly-tests.sh | 8 + .../test-scripts/test_batch_allround.sh | 36 +++ 5 files changed, 434 insertions(+) create mode 100644 flink-end-to-end-tests/flink-dataset-allround-test/pom.xml create mode 100644 flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java create mode 100755 flink-end-to-end-tests/test-scripts/test_batch_allround.sh diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml new file mode 100644 index 0000000000000..b701dfdeeb8aa --- /dev/null +++ b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml @@ -0,0 +1,104 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-end-to-end-tests + 1.6-SNAPSHOT + .. + + + flink-dataset-allround-test + flink-dataset-allround-test + jar + + + + org.apache.flink + flink-core + ${project.version} + + + org.apache.flink + flink-java + ${project.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + DataSetAllroundTestProgram + package + + jar + + + DataSetAllroundTestProgram + + + + org.apache.flink.batch.tests.DataSetAllroundTestProgram + + + + + org/apache/flink/batch/tests/DataSetAllroundTestProgram.class + org/apache/flink/batch/tests/DataSetAllroundTestProgram$*.class + + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + 1.7 + + + rename + package + + run + + + + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java b/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java new file mode 100644 index 0000000000000..039753545b59e --- /dev/null +++ b/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java @@ -0,0 +1,285 @@ +/* + * 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.batch.tests; + +import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.api.common.operators.Order; +import org.apache.flink.api.common.operators.base.JoinOperatorBase; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.DataSet; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.operators.IterativeDataSet; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.io.GenericInputSplit; +import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +/** + * Program to test a large chunk of DataSet API operators and primitives: + *

      + *
    • Map, FlatMap, Filter
    • + *
    • GroupReduce, Reduce
    • + *
    • Join
    • + *
    • CoGroup
    • + *
    • BulkIteration
    • + *
    • Different key definitions (position, name, KeySelector)
    • + *
    + * + *

    Program parameters: + *

      + *
    • loadFactor (int): controls generated data volume. Does not affect result.
    • + *
    • outputPath (String): path to write the result
    • + *
    + */ +public class DataSetAllroundTestProgram { + + public static void main(String[] args) throws Exception { + + // get parameters + ParameterTool params = ParameterTool.fromArgs(args); + int loadFactor = Integer.parseInt(params.getRequired("loadFactor")); + String outputPath = params.getRequired("outputPath"); + + ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + + int numKeys = loadFactor * 128 * 1024; + DataSet> x1Keys = env.createInput(new Generator(numKeys, 1)).setParallelism(4); + DataSet> x2Keys = env.createInput(new Generator(numKeys * 32, 2)).setParallelism(4); + DataSet> x8Keys = env.createInput(new Generator(numKeys, 8)).setParallelism(4); + + DataSet> joined = x2Keys + // shift keys (check for correct handling of key positions) + .map(x -> Tuple4.of("0-0", 0L, 1, x.f0)) + .returns(Types.TUPLE(Types.STRING, Types.LONG, Types.INT, Types.STRING)) + // join datasets on non-unique fields (m-n join) + // Result: (key, 1) 16 * #keys records, all keys are preserved + .join(x8Keys).where(3).equalTo(0).with((l, r) -> Tuple2.of(l.f3, 1)) + .returns(Types.TUPLE(Types.STRING, Types.INT)) + // key definition with key selector function + .groupBy( + new KeySelector, String>() { + @Override + public String getKey(Tuple2 value) throws Exception { + return value.f0; + } + } + ) + // reduce + // Result: (key, cnt), #keys records with unique keys, cnt = 16 + .reduce((value1, value2) -> Tuple2.of(value1.f0, value1.f1 + value2.f1)); + + // co-group two datasets on their primary keys. + // we filter both inputs such that only 6.25% of the keys overlap. + // result: (key, cnt), #keys records with unique keys, cnt = (6.25%: 2, 93.75%: 1) + DataSet> coGrouped = x1Keys + .filter(x -> x.f1 > 59) + .coGroup(x1Keys.filter(x -> x.f1 < 68)).where("f0").equalTo("f0").with( + (CoGroupFunction, Tuple2, Tuple2>) + (l, r, out) -> { + int cnt = 0; + String key = ""; + for (Tuple2 t : l) { + cnt++; + key = t.f0; + } + for (Tuple2 t : r) { + cnt++; + key = t.f0; + } + out.collect(Tuple2.of(key, cnt)); + } + ) + .returns(Types.TUPLE(Types.STRING, Types.INT)); + + // join datasets on keys (1-1 join) and replicate by 16 (previously computed count) + // result: (key, cnt), 16 * #keys records, all keys preserved, cnt = (6.25%: 2, 93.75%: 1) + DataSet> joined2 = joined.join(coGrouped, JoinOperatorBase.JoinHint.REPARTITION_SORT_MERGE) + .where(0).equalTo("f0") + .flatMap( + (FlatMapFunction, Tuple2>, Tuple2>) + (p, out) -> { + for (int i = 0; i < p.f0.f1; i++) { + out.collect(Tuple2.of(p.f0.f0, p.f1.f1)); + } + } + ) + .returns(Types.TUPLE(Types.STRING, Types.INT)); + + // iteration. double the count field until all counts are at 32 or more + // result: (key, cnt), 16 * #keys records, all keys preserved, cnt = (6.25%: 64, 93.75%: 32) + IterativeDataSet> initial = joined2.iterate(16); + DataSet> iteration = initial + .map(x -> Tuple2.of(x.f0, x.f1 * 2)) + .returns(Types.TUPLE(Types.STRING, Types.INT)); + DataSet termination = iteration + // stop iteration if all values are larger/equal 32 + .flatMap( + (FlatMapFunction, Boolean>) + (x, out) -> { + if (x.f1 < 32) { + out.collect(false); + } + } + ) + .returns(Types.BOOLEAN); + DataSet result = initial.closeWith(iteration, termination) + // group on the count field and count records + // result: two records: (32, cnt1) and (64, cnt2) where cnt1 = x * 15/16, cnt2 = x * 1/16 + .groupBy(1) + .reduceGroup( + (GroupReduceFunction, Tuple2>) + (g, out) -> { + int key = 0; + int cnt = 0; + for (Tuple2 r : g) { + key = r.f1; + cnt++; + } + out.collect(Tuple2.of(key, cnt)); + } + ) + .returns(Types.TUPLE(Types.INT, Types.INT)) + // normalize result by load factor + // result: two records: (32: 15360) and (64, 1024). (x = 16384) + .map(x -> Tuple2.of(x.f0, x.f1 / (loadFactor * 128))) + .returns(Types.TUPLE(Types.INT, Types.INT)); + + // sort and emit result + result + .sortPartition(0, Order.ASCENDING).setParallelism(1) + .writeAsText(outputPath, FileSystem.WriteMode.OVERWRITE).setParallelism(1); + + env.execute(); + + } + + /** + * InputFormat that generates a deterministic DataSet of Tuple2(String, Integer) + *
      + *
    • String: key, can be repeated.
    • + *
    • Integer: uniformly distributed int between 0 and 127
    • + *
    + */ + public static class Generator implements InputFormat, GenericInputSplit> { + + // total number of records + private final long numRecords; + // total number of keys + private final long numKeys; + + // records emitted per partition + private long recordsPerPartition; + // number of keys per partition + private long keysPerPartition; + + // number of currently emitted records + private long recordCnt; + + // id of current partition + private int partitionId; + // total number of partitions + private int numPartitions; + + public Generator(long numKeys, int recordsPerKey) { + this.numKeys = numKeys; + this.numRecords = numKeys * recordsPerKey; + } + + @Override + public void configure(Configuration parameters) { } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { + return null; + } + + @Override + public GenericInputSplit[] createInputSplits(int minNumSplits) throws IOException { + + GenericInputSplit[] splits = new GenericInputSplit[minNumSplits]; + for (int i = 0; i < minNumSplits; i++) { + splits[i] = new GenericInputSplit(i, minNumSplits); + } + return splits; + } + + @Override + public InputSplitAssigner getInputSplitAssigner(GenericInputSplit[] inputSplits) { + return new DefaultInputSplitAssigner(inputSplits); + } + + @Override + public void open(GenericInputSplit split) throws IOException { + this.partitionId = split.getSplitNumber(); + this.numPartitions = split.getTotalNumberOfSplits(); + + // ensure even distribution of records and keys + Preconditions.checkArgument( + numRecords % numPartitions == 0, + "Records cannot be evenly distributed among partitions"); + Preconditions.checkArgument( + numKeys % numPartitions == 0, + "Keys cannot be evenly distributed among partitions"); + + this.recordsPerPartition = numRecords / numPartitions; + this.keysPerPartition = numKeys / numPartitions; + + this.recordCnt = 0; + } + + @Override + public boolean reachedEnd() throws IOException { + return this.recordCnt >= this.recordsPerPartition; + } + + @Override + public Tuple2 nextRecord(Tuple2 reuse) throws IOException { + + // build key from partition id and count per partition + String key = String.format( + "%d-%d", + this.partitionId, + this.recordCnt % this.keysPerPartition); + // 128 values to filter on + int filterVal = (int) this.recordCnt % 128; + + this.recordCnt++; + + reuse.f0 = key; + reuse.f1 = filterVal; + return reuse; + } + + @Override + public void close() throws IOException { } + } + +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 695a4f1324d85..db86cc1854b1c 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -36,6 +36,7 @@ under the License. flink-parent-child-classloading-test + flink-dataset-allround-test diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 8ee526bc23ed0..71224e09f1b7f 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -47,5 +47,13 @@ EXIT_CODE=0 # EXIT_CODE=$? # fi +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running DataSet allround nightly end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_batch_allround.sh + EXIT_CODE=$? +fi + # Exit code for Travis build success/failure exit $EXIT_CODE diff --git a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh new file mode 100755 index 0000000000000..1e05c7f044648 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +source "$(dirname "$0")"/common.sh + +TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-dataset-allround-test/target/DataSetAllroundTestProgram.jar + + echo "Run DataSet-Allround-Test Program" + +start_cluster +$FLINK_DIR/bin/taskmanager.sh start +$FLINK_DIR/bin/taskmanager.sh start +$FLINK_DIR/bin/taskmanager.sh start + +$FLINK_DIR/bin/flink run -p 4 $TEST_PROGRAM_JAR --loadFactor 2 --outputPath $TEST_DATA_DIR/out/dataset_allround + +stop_cluster +$FLINK_DIR/bin/taskmanager.sh stop-all + +check_result_hash "DataSet-Allround-Test" $TEST_DATA_DIR/out/dataset_allround "d3cf2aeaa9320c772304cba42649eb47" From 69d98a3253abc478a03244db27e17788f77fe840 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 26 Mar 2018 15:55:04 +0200 Subject: [PATCH 217/268] [FLINK-8972] [e2eTests] Remove IDE warnings and simplify pom.xml This closes #5752. --- .../flink-dataset-allround-test/pom.xml | 25 ++----------------- .../tests/DataSetAllroundTestProgram.java | 20 +++++++-------- .../test-scripts/test_batch_allround.sh | 2 +- 3 files changed, 12 insertions(+), 35 deletions(-) diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml index b701dfdeeb8aa..b24774393a83a 100644 --- a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml +++ b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.6-SNAPSHOT + 1.5-SNAPSHOT .. @@ -61,7 +61,7 @@ under the License. jar - DataSetAllroundTestProgram + DataSetAllroundTestProgram @@ -77,27 +77,6 @@ under the License. - - - - org.apache.maven.plugins - maven-antrun-plugin - 1.7 - - - rename - package - - run - - - - - - - - -
    diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java b/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java index 039753545b59e..4abbb3548b0eb 100644 --- a/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java +++ b/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java @@ -40,8 +40,6 @@ import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.util.Preconditions; -import java.io.IOException; - /** * Program to test a large chunk of DataSet API operators and primitives: *
      @@ -61,6 +59,7 @@ */ public class DataSetAllroundTestProgram { + @SuppressWarnings("Convert2Lambda") public static void main(String[] args) throws Exception { // get parameters @@ -87,7 +86,7 @@ public static void main(String[] args) throws Exception { .groupBy( new KeySelector, String>() { @Override - public String getKey(Tuple2 value) throws Exception { + public String getKey(Tuple2 value) { return value.f0; } } @@ -150,7 +149,7 @@ public String getKey(Tuple2 value) throws Exception { } ) .returns(Types.BOOLEAN); - DataSet result = initial.closeWith(iteration, termination) + DataSet> result = initial.closeWith(iteration, termination) // group on the count field and count records // result: two records: (32, cnt1) and (64, cnt2) where cnt1 = x * 15/16, cnt2 = x * 1/16 .groupBy(1) @@ -178,7 +177,6 @@ public String getKey(Tuple2 value) throws Exception { .writeAsText(outputPath, FileSystem.WriteMode.OVERWRITE).setParallelism(1); env.execute(); - } /** @@ -217,12 +215,12 @@ public Generator(long numKeys, int recordsPerKey) { public void configure(Configuration parameters) { } @Override - public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOException { + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { return null; } @Override - public GenericInputSplit[] createInputSplits(int minNumSplits) throws IOException { + public GenericInputSplit[] createInputSplits(int minNumSplits) { GenericInputSplit[] splits = new GenericInputSplit[minNumSplits]; for (int i = 0; i < minNumSplits; i++) { @@ -237,7 +235,7 @@ public InputSplitAssigner getInputSplitAssigner(GenericInputSplit[] inputSplits) } @Override - public void open(GenericInputSplit split) throws IOException { + public void open(GenericInputSplit split) { this.partitionId = split.getSplitNumber(); this.numPartitions = split.getTotalNumberOfSplits(); @@ -256,12 +254,12 @@ public void open(GenericInputSplit split) throws IOException { } @Override - public boolean reachedEnd() throws IOException { + public boolean reachedEnd() { return this.recordCnt >= this.recordsPerPartition; } @Override - public Tuple2 nextRecord(Tuple2 reuse) throws IOException { + public Tuple2 nextRecord(Tuple2 reuse) { // build key from partition id and count per partition String key = String.format( @@ -279,7 +277,7 @@ public Tuple2 nextRecord(Tuple2 reuse) throws } @Override - public void close() throws IOException { } + public void close() { } } } diff --git a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh index 1e05c7f044648..acdc37e685fda 100755 --- a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh +++ b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh @@ -21,7 +21,7 @@ source "$(dirname "$0")"/common.sh TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-dataset-allround-test/target/DataSetAllroundTestProgram.jar - echo "Run DataSet-Allround-Test Program" +echo "Run DataSet-Allround-Test Program" start_cluster $FLINK_DIR/bin/taskmanager.sh start From 337b4ef23ac9b980935da04e714640f7d670221e Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 14 Mar 2018 18:11:34 +0100 Subject: [PATCH 218/268] [FLINK-8833] [sql-client] Create a SQL Client JSON format fat-jar This closes #5700. --- flink-formats/flink-json/pom.xml | 33 ++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/flink-formats/flink-json/pom.xml b/flink-formats/flink-json/pom.xml index 1c77a4535b14f..f7332b9d8be58 100644 --- a/flink-formats/flink-json/pom.xml +++ b/flink-formats/flink-json/pom.xml @@ -63,6 +63,13 @@ under the License. + + org.apache.flink + flink-test-utils-junit + ${project.version} + test + + org.apache.flink @@ -79,4 +86,30 @@ under the License. test + + + + + release + + + + org.apache.maven.plugins + maven-jar-plugin + + + package + + jar + + + sql-jar + + + + + + + + From e2285ad5075397198ed2434e6161dcfb582d32ad Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 26 Mar 2018 19:03:28 +0200 Subject: [PATCH 219/268] [hotfix] [sql-client] Add flink-sql-client as 'provided' to flink-dist --- flink-dist/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index 90e79ea155df5..79eb53fe2621d 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -308,6 +308,13 @@ under the License. provided + + org.apache.flink + flink-sql-client + ${project.version} + provided + + org.apache.flink flink-s3-fs-hadoop From 1cb9b73951453b109ed7579429e6a9a1a3962994 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Mar 2018 11:29:50 +0100 Subject: [PATCH 220/268] [FLINK-8789] Throw IllegalStateException when calling Task#stopExecution on not running Task Before we threw an UnsupportedOperationException when calling Task#stopExecution on a Task where we did not set the invokable yet. This can be a bit misleading and, thus, this commit throws an IllegalStateException with an respective message. This closes #5753. --- .../flink/runtime/taskmanager/Task.java | 45 +++++++++++-------- .../runtime/taskmanager/TaskManagerTest.java | 4 ++ 2 files changed, 30 insertions(+), 19 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index ccb850e85bc12..d99472bb6b6ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -628,8 +628,7 @@ else if (current == ExecutionState.CANCELING) { // next, kick off the background copying of files for the distributed cache try { for (Map.Entry entry : - DistributedCache.readFileInfoFromConfig(jobConfiguration)) - { + DistributedCache.readFileInfoFromConfig(jobConfiguration)) { LOG.info("Obtaining local cache file for '{}'.", entry.getKey()); Future cp = fileCache.createTmpFile(entry.getKey(), entry.getValue(), jobId); distributedCacheEntries.put(entry.getKey(), cp); @@ -943,26 +942,34 @@ private boolean transitionState(ExecutionState currentState, ExecutionState newS * This method never blocks. *

      * - * @throws UnsupportedOperationException - * if the {@link AbstractInvokable} does not implement {@link StoppableTask} + * @throws UnsupportedOperationException if the {@link AbstractInvokable} does not implement {@link StoppableTask} + * @throws IllegalStateException if the {@link Task} is not yet running */ - public void stopExecution() throws UnsupportedOperationException { - LOG.info("Attempting to stop task {} ({}).", taskNameWithSubtask, executionId); - if (invokable instanceof StoppableTask) { - Runnable runnable = new Runnable() { - @Override - public void run() { - try { - ((StoppableTask) invokable).stop(); - } catch (RuntimeException e) { - LOG.error("Stopping task {} ({}) failed.", taskNameWithSubtask, executionId, e); - taskManagerActions.failTask(executionId, e); + public void stopExecution() { + if (invokable != null) { + LOG.info("Attempting to stop task {} ({}).", taskNameWithSubtask, executionId); + if (invokable instanceof StoppableTask) { + Runnable runnable = new Runnable() { + @Override + public void run() { + try { + ((StoppableTask) invokable).stop(); + } catch (RuntimeException e) { + LOG.error("Stopping task {} ({}) failed.", taskNameWithSubtask, executionId, e); + taskManagerActions.failTask(executionId, e); + } } - } - }; - executeAsyncCallRunnable(runnable, String.format("Stopping source task %s (%s).", taskNameWithSubtask, executionId)); + }; + executeAsyncCallRunnable(runnable, String.format("Stopping source task %s (%s).", taskNameWithSubtask, executionId)); + } else { + throw new UnsupportedOperationException(String.format("Stopping not supported by task %s (%s).", taskNameWithSubtask, executionId)); + } } else { - throw new UnsupportedOperationException(String.format("Stopping not supported by task %s (%s).", taskNameWithSubtask, executionId)); + throw new IllegalStateException( + String.format( + "Cannot stop task %s (%s) because it is not yet running.", + taskNameWithSubtask, + executionId)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 93ec3adcb2a42..977ab9ed4fb14 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -1734,6 +1734,10 @@ public void testStopTaskFailure() throws Exception { Await.result(submitResponse, timeout); + final Future taskRunning = taskManager.ask(new TestingTaskManagerMessages.NotifyWhenTaskIsRunning(executionAttemptId), timeout); + + Await.result(taskRunning, timeout); + Future stopResponse = taskManager.ask(new StopTask(executionAttemptId), timeout); try { From 946902e4bc40a56a1656e63eb69ca9fd999dcbf1 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Mar 2018 12:01:09 +0100 Subject: [PATCH 221/268] [FLINK-8901] [yarn] Set proper Yarn application name When deploying a session cluster, Flink will register under "Flink session cluster". When deploying a per-job cluster, Flink will register under "Flink per-job cluster". This closes #5754. --- .../yarn/AbstractYarnClusterDescriptor.java | 17 +++++++---------- .../flink/yarn/Flip6YarnClusterDescriptor.java | 1 + 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index caf7a7614c7f2..c80818a2e4b51 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -394,6 +394,7 @@ public ClusterClient deploySessionCluster(ClusterSpecification cl try { return deployInternal( clusterSpecification, + "Flink session cluster", getYarnSessionClusterEntrypoint(), null, false); @@ -437,12 +438,14 @@ private void validateClusterSpecification(ClusterSpecification clusterSpecificat * This method will block until the ApplicationMaster/JobManager have been deployed on YARN. * * @param clusterSpecification Initial cluster specification for the Flink cluster to be deployed + * @param applicationName name of the Yarn application to start * @param yarnClusterEntrypoint Class name of the Yarn cluster entry point. * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none * @param detached True if the cluster should be started in detached mode */ protected ClusterClient deployInternal( ClusterSpecification clusterSpecification, + String applicationName, String yarnClusterEntrypoint, @Nullable JobGraph jobGraph, boolean detached) throws Exception { @@ -517,6 +520,7 @@ protected ClusterClient deployInternal( ApplicationReport report = startAppMaster( flinkConfiguration, + applicationName, yarnClusterEntrypoint, jobGraph, yarnClient, @@ -670,6 +674,7 @@ private void checkYarnQueues(YarnClient yarnClient) { public ApplicationReport startAppMaster( Configuration configuration, + String applicationName, String yarnClusterEntrypoint, JobGraph jobGraph, YarnClient yarnClient, @@ -1005,17 +1010,9 @@ public ApplicationReport startAppMaster( capability.setMemory(clusterSpecification.getMasterMemoryMB()); capability.setVirtualCores(1); - String name; - if (customName == null) { - name = "Flink session with " + clusterSpecification.getNumberTaskManagers() + " TaskManagers"; - if (detached) { - name += " (detached)"; - } - } else { - name = customName; - } + final String customApplicationName = customName != null ? customName : applicationName; - appContext.setApplicationName(name); + appContext.setApplicationName(customApplicationName); appContext.setApplicationType("Apache Flink"); appContext.setAMContainerSpec(amContainer); appContext.setResource(capability); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java index 9860363c00e32..1374ca2c4419d 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java @@ -75,6 +75,7 @@ public ClusterClient deployJobCluster( try { return deployInternal( clusterSpecification, + "Flink per-job cluster", getYarnJobClusterEntrypoint(), jobGraph, detached); From a9b52758db49ccbbebb83f41736d84388cd35d6c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 27 Mar 2018 08:05:32 +0200 Subject: [PATCH 222/268] [hotfix] Remove unused applicationName parameter from FlinkYarnSessionCli#createDescriptor --- .../flink/yarn/cli/FlinkYarnSessionCli.java | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 2311e875c2f13..446377ffab728 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -265,11 +265,10 @@ public FlinkYarnSessionCli( } private AbstractYarnClusterDescriptor createDescriptor( - Configuration configuration, - YarnConfiguration yarnConfiguration, - String configurationDirectory, - String defaultApplicationName, - CommandLine cmd) { + Configuration configuration, + YarnConfiguration yarnConfiguration, + String configurationDirectory, + CommandLine cmd) { AbstractYarnClusterDescriptor yarnClusterDescriptor = getClusterDescriptor( configuration, @@ -356,11 +355,6 @@ private AbstractYarnClusterDescriptor createDescriptor( if (cmd.hasOption(name.getOpt())) { yarnClusterDescriptor.setName(cmd.getOptionValue(name.getOpt())); - } else { - // set the default application name, if none is specified - if (defaultApplicationName != null) { - yarnClusterDescriptor.setName(defaultApplicationName); - } } if (cmd.hasOption(zookeeperNamespace.getOpt())) { @@ -456,7 +450,6 @@ public AbstractYarnClusterDescriptor createClusterDescriptor(CommandLine command effectiveConfiguration, yarnConfiguration, configurationDirectory, - null, commandLine); } From 529494386c7b4a899f66f37b711c1f2581047611 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 23 Mar 2018 16:21:55 +0100 Subject: [PATCH 223/268] [FLINK-5411] [flip6] Fix JobLeaderIdService shut down in ResourceManager The JobLeaderIdService was formerly closed at two different locations. Once in the ResourceManager and once in the ResourceManagerRuntimeServices. Since the JobLeaderIdService is a RM specific component. It should also be closed in the scope of the RM. This closes #5757. --- .../runtime/resourcemanager/ResourceManagerRunner.java | 9 +-------- .../resourcemanager/ResourceManagerRuntimeServices.java | 6 ------ 2 files changed, 1 insertion(+), 14 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java index 9daf96ef48a98..ff9b4f0789e55 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRunner.java @@ -20,7 +20,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -106,13 +105,7 @@ public CompletableFuture closeAsync() { synchronized (lock) { resourceManager.shutDown(); - return FutureUtils.runAfterwards( - resourceManager.getTerminationFuture(), - () -> { - synchronized (lock) { - resourceManagerRuntimeServices.shutDown(); - } - }); + return resourceManager.getTerminationFuture(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java index ed8f1e0cd9689..7f5af2445930e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerRuntimeServices.java @@ -45,12 +45,6 @@ public JobLeaderIdService getJobLeaderIdService() { return jobLeaderIdService; } - // -------------------- Lifecycle methods ----------------------------------- - - public void shutDown() throws Exception { - jobLeaderIdService.stop(); - } - // -------------------- Static methods -------------------------------------- public static ResourceManagerRuntimeServices fromConfiguration( From b466eb2955ce353072515642de8597c569cbbce0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 25 Mar 2018 19:12:51 +0200 Subject: [PATCH 224/268] [FLINK-8940] [flip6] Add support for dispose savepoint Adds an AsynchronousOperationHandler for disposing savepoints. The handler is registered under '/savepoint-disposal' and requires a SavepointDisposalRequest JSON object containing the path to the savepoint to be disposed. The RestClusterClient polls the status registered under '/savepoint-disposal/:triggerId' until the operation has been completed. This closes #5764. --- .../apache/flink/client/cli/CliFrontend.java | 3 +- .../flink/client/program/ClusterClient.java | 4 +- .../client/program/MiniClusterClient.java | 4 +- .../program/rest/RestClusterClient.java | 38 ++++++ .../client/cli/CliFrontendSavepointTest.java | 17 ++- .../client/program/ClusterClientTest.java | 4 +- .../program/rest/RestClusterClientTest.java | 107 +++++++++++++++++ .../flink/runtime/checkpoint/Checkpoints.java | 11 +- .../flink/runtime/dispatcher/Dispatcher.java | 21 ++++ .../runtime/minicluster/MiniCluster.java | 12 ++ .../rest/handler/async/OperationKey.java | 4 +- .../savepoints/SavepointDisposalHandlers.java | 112 ++++++++++++++++++ .../savepoints/SavepointDisposalRequest.java | 49 ++++++++ .../SavepointDisposalStatusHeaders.java | 75 ++++++++++++ ...epointDisposalStatusMessageParameters.java | 46 +++++++ .../SavepointDisposalTriggerHeaders.java | 67 +++++++++++ .../runtime/webmonitor/RestfulGateway.java | 13 ++ .../webmonitor/WebMonitorEndpoint.java | 19 +++ .../runtime/dispatcher/DispatcherTest.java | 63 +++++++++- .../SavepointDisposalRequestTest.java | 47 ++++++++ 20 files changed, 691 insertions(+), 25 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointDisposalHandlers.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequest.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusMessageParameters.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalTriggerHeaders.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequestTest.java diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index d636ef7c9615e..a874891eb25c7 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -47,7 +47,6 @@ import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.messages.Acknowledge; @@ -706,7 +705,7 @@ private void disposeSavepoint(ClusterClient clusterClient, String savepointPa logAndSysout("Disposing savepoint '" + savepointPath + "'."); - final CompletableFuture disposeFuture = clusterClient.disposeSavepoint(savepointPath, FutureUtils.toTime(clientTimeout)); + final CompletableFuture disposeFuture = clusterClient.disposeSavepoint(savepointPath); logAndSysout("Waiting for response..."); diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java index f50206d1492c4..fbaa5156520a3 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ClusterClient.java @@ -726,14 +726,14 @@ public CompletableFuture triggerSavepoint(JobID jobId, @Nullable String }); } - public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) throws FlinkException { + public CompletableFuture disposeSavepoint(String savepointPath) throws FlinkException { final ActorGateway jobManager = getJobManagerGateway(); Object msg = new JobManagerMessages.DisposeSavepoint(savepointPath); CompletableFuture responseFuture = FutureUtils.toJava( jobManager.ask( msg, - FutureUtils.toFiniteDuration(timeout))); + timeout)); return responseFuture.thenApply( (Object response) -> { diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java index 44f6ef630d26a..802622ee4ae0c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/MiniClusterClient.java @@ -123,8 +123,8 @@ public CompletableFuture triggerSavepoint(JobID jobId, @Nullable String } @Override - public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) throws FlinkException { - throw new UnsupportedOperationException("MiniClusterClient does not yet support this operation."); + public CompletableFuture disposeSavepoint(String savepointPath) throws FlinkException { + return guardWithSingleRetry(() -> miniCluster.disposeSavepoint(savepointPath), scheduledExecutor); } @Override diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 2e1ffb0229818..912971456f668 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -72,6 +72,10 @@ import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders; import org.apache.flink.runtime.rest.messages.job.JobSubmitRequestBody; import org.apache.flink.runtime.rest.messages.job.JobSubmitResponseBody; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalRequest; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusHeaders; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusMessageParameters; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalTriggerHeaders; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointInfo; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointStatusHeaders; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointStatusMessageParameters; @@ -530,6 +534,40 @@ public CompletableFuture rescaleJob(JobID jobId, int newParallelism }); } + @Override + public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) { + final SavepointDisposalRequest savepointDisposalRequest = new SavepointDisposalRequest(savepointPath); + + final CompletableFuture savepointDisposalTriggerFuture = sendRequest( + SavepointDisposalTriggerHeaders.getInstance(), + EmptyMessageParameters.getInstance(), + savepointDisposalRequest); + + final CompletableFuture savepointDisposalFuture = savepointDisposalTriggerFuture.thenCompose( + (TriggerResponse triggerResponse) -> { + final TriggerId triggerId = triggerResponse.getTriggerId(); + final SavepointDisposalStatusHeaders savepointDisposalStatusHeaders = SavepointDisposalStatusHeaders.getInstance(); + final SavepointDisposalStatusMessageParameters savepointDisposalStatusMessageParameters = savepointDisposalStatusHeaders.getUnresolvedMessageParameters(); + savepointDisposalStatusMessageParameters.triggerIdPathParameter.resolve(triggerId); + + return pollResourceAsync( + () -> sendRetryableRequest( + savepointDisposalStatusHeaders, + savepointDisposalStatusMessageParameters, + EmptyRequestBody.getInstance(), + isConnectionProblemException())); + }); + + return savepointDisposalFuture.thenApply( + (AsynchronousOperationInfo asynchronousOperationInfo) -> { + if (asynchronousOperationInfo.getFailureCause() == null) { + return Acknowledge.get(); + } else { + throw new CompletionException(asynchronousOperationInfo.getFailureCause()); + } + }); + } + /** * Creates a {@code CompletableFuture} that polls a {@code AsynchronouslyCreatedResource} until * its {@link AsynchronouslyCreatedResource#queueStatus() QueueStatus} becomes diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java index 3195a6baf6dc1..75f3b3df18439 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java @@ -19,7 +19,6 @@ package org.apache.flink.client.cli; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Time; import org.apache.flink.client.cli.util.MockedCliFrontend; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.StandaloneClusterClient; @@ -41,7 +40,7 @@ import java.io.FileOutputStream; import java.io.PrintStream; import java.util.concurrent.CompletableFuture; -import java.util.function.BiFunction; +import java.util.function.Function; import java.util.zip.ZipOutputStream; import static org.junit.Assert.assertEquals; @@ -196,7 +195,7 @@ public void testDisposeSavepointSuccess() throws Exception { String savepointPath = "expectedSavepointPath"; ClusterClient clusterClient = new DisposeSavepointClusterClient( - (String path, Time timeout) -> CompletableFuture.completedFuture(Acknowledge.get()), getConfiguration()); + (String path) -> CompletableFuture.completedFuture(Acknowledge.get()), getConfiguration()); try { @@ -225,7 +224,7 @@ public void testDisposeWithJar() throws Exception { final CompletableFuture disposeSavepointFuture = new CompletableFuture<>(); final DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient( - (String savepointPath, Time timeout) -> { + (String savepointPath) -> { disposeSavepointFuture.complete(savepointPath); return CompletableFuture.completedFuture(Acknowledge.get()); }, getConfiguration()); @@ -260,7 +259,7 @@ public void testDisposeSavepointFailure() throws Exception { Exception testException = new Exception("expectedTestException"); - DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient((String path, Time timeout) -> FutureUtils.completedExceptionally(testException), getConfiguration()); + DisposeSavepointClusterClient clusterClient = new DisposeSavepointClusterClient((String path) -> FutureUtils.completedExceptionally(testException), getConfiguration()); try { CliFrontend frontend = new MockedCliFrontend(clusterClient); @@ -285,17 +284,17 @@ public void testDisposeSavepointFailure() throws Exception { private static final class DisposeSavepointClusterClient extends StandaloneClusterClient { - private final BiFunction> disposeSavepointFunction; + private final Function> disposeSavepointFunction; - DisposeSavepointClusterClient(BiFunction> disposeSavepointFunction, Configuration configuration) { + DisposeSavepointClusterClient(Function> disposeSavepointFunction, Configuration configuration) { super(configuration, new TestingHighAvailabilityServices(), false); this.disposeSavepointFunction = Preconditions.checkNotNull(disposeSavepointFunction); } @Override - public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) { - return disposeSavepointFunction.apply(savepointPath, timeout); + public CompletableFuture disposeSavepoint(String savepointPath) { + return disposeSavepointFunction.apply(savepointPath); } } diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java index f30fd192e8388..07b3821ce2afc 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClusterClientTest.java @@ -181,7 +181,7 @@ public void testDisposeSavepointUnknownResponse() throws Exception { final TestClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway); - CompletableFuture acknowledgeCompletableFuture = clusterClient.disposeSavepoint(savepointPath, timeout); + CompletableFuture acknowledgeCompletableFuture = clusterClient.disposeSavepoint(savepointPath); try { acknowledgeCompletableFuture.get(); @@ -203,7 +203,7 @@ public void testDisposeClassNotFoundException() throws Exception { final TestClusterClient clusterClient = new TestClusterClient(configuration, jobManagerGateway); - CompletableFuture acknowledgeCompletableFuture = clusterClient.disposeSavepoint(savepointPath, timeout); + CompletableFuture acknowledgeCompletableFuture = clusterClient.disposeSavepoint(savepointPath); try { acknowledgeCompletableFuture.get(); diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index 77a4113f59ed4..926da924c5cd3 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.webmonitor.JobDetails; import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails; import org.apache.flink.runtime.rest.RestClient; @@ -45,6 +46,7 @@ import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.handler.RestHandlerException; import org.apache.flink.runtime.rest.handler.RestHandlerSpecification; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationInfo; import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationResult; import org.apache.flink.runtime.rest.handler.async.TriggerResponse; import org.apache.flink.runtime.rest.messages.AccumulatorsIncludeSerializedValueQueryParameter; @@ -72,6 +74,10 @@ import org.apache.flink.runtime.rest.messages.job.JobSubmitHeaders; import org.apache.flink.runtime.rest.messages.job.JobSubmitRequestBody; import org.apache.flink.runtime.rest.messages.job.JobSubmitResponseBody; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalRequest; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusHeaders; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusMessageParameters; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalTriggerHeaders; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointInfo; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointStatusHeaders; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointStatusMessageParameters; @@ -84,7 +90,9 @@ import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.testutils.category.Flip6; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.OptionalFailure; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; @@ -103,6 +111,7 @@ import javax.annotation.Nonnull; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -112,6 +121,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -536,6 +546,103 @@ protected CompletableFuture> handleRe } } + @Test + public void testDisposeSavepoint() throws Exception { + final String savepointPath = "foobar"; + final String exceptionMessage = "Test exception."; + final FlinkException testException = new FlinkException(exceptionMessage); + + final TestSavepointDisposalHandlers testSavepointDisposalHandlers = new TestSavepointDisposalHandlers(savepointPath); + final TestSavepointDisposalHandlers.TestSavepointDisposalTriggerHandler testSavepointDisposalTriggerHandler = testSavepointDisposalHandlers.new TestSavepointDisposalTriggerHandler(); + final TestSavepointDisposalHandlers.TestSavepointDisposalStatusHandler testSavepointDisposalStatusHandler = testSavepointDisposalHandlers.new TestSavepointDisposalStatusHandler( + OptionalFailure.of(AsynchronousOperationInfo.complete()), + OptionalFailure.of(AsynchronousOperationInfo.completeExceptional(new SerializedThrowable(testException))), + OptionalFailure.ofFailure(testException)); + + try (TestRestServerEndpoint ignored = createRestServerEndpoint( + testSavepointDisposalStatusHandler, + testSavepointDisposalTriggerHandler)) { + { + final CompletableFuture disposeSavepointFuture = restClusterClient.disposeSavepoint(savepointPath); + assertThat(disposeSavepointFuture.get(), is(Acknowledge.get())); + } + + { + final CompletableFuture disposeSavepointFuture = restClusterClient.disposeSavepoint(savepointPath); + + try { + disposeSavepointFuture.get(); + fail("Expected an exception"); + } catch (ExecutionException ee) { + assertThat(ExceptionUtils.findThrowableWithMessage(ee, exceptionMessage).isPresent(), is(true)); + } + } + + { + try { + restClusterClient.disposeSavepoint(savepointPath).get(); + fail("Expected an exception."); + } catch (ExecutionException ee) { + assertThat(ExceptionUtils.findThrowable(ee, RestClientException.class).isPresent(), is(true)); + } + } + } + } + + private class TestSavepointDisposalHandlers { + + private final TriggerId triggerId = new TriggerId(); + + private final String savepointPath; + + private TestSavepointDisposalHandlers(String savepointPath) { + this.savepointPath = Preconditions.checkNotNull(savepointPath); + } + + private class TestSavepointDisposalTriggerHandler extends TestHandler { + private TestSavepointDisposalTriggerHandler() { + super(SavepointDisposalTriggerHeaders.getInstance()); + } + + @Override + protected CompletableFuture handleRequest(@Nonnull HandlerRequest request, @Nonnull DispatcherGateway gateway) { + assertThat(request.getRequestBody().getSavepointPath(), is(savepointPath)); + return CompletableFuture.completedFuture(new TriggerResponse(triggerId)); + } + } + + private class TestSavepointDisposalStatusHandler extends TestHandler, SavepointDisposalStatusMessageParameters> { + + private final Queue> responses; + + private TestSavepointDisposalStatusHandler(OptionalFailure... responses) { + super(SavepointDisposalStatusHeaders.getInstance()); + this.responses = new ArrayDeque<>(Arrays.asList(responses)); + } + + @Override + protected CompletableFuture> handleRequest(@Nonnull HandlerRequest request, @Nonnull DispatcherGateway gateway) throws RestHandlerException { + final TriggerId actualTriggerId = request.getPathParameter(TriggerIdPathParameter.class); + + if (actualTriggerId.equals(triggerId)) { + final OptionalFailure nextResponse = responses.poll(); + + if (nextResponse != null) { + if (nextResponse.isFailure()) { + throw new RestHandlerException("Failure", HttpResponseStatus.BAD_REQUEST, nextResponse.getFailureCause()); + } else { + return CompletableFuture.completedFuture(AsynchronousOperationResult.completed(nextResponse.getUnchecked())); + } + } else { + throw new AssertionError(); + } + } else { + throw new AssertionError(); + } + } + } + } + @Test public void testListJobs() throws Exception { try (TestRestServerEndpoint ignored = createRestServerEndpoint(new TestListJobsHandler())) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java index 72b7c53ab9589..60fdc17c819de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/Checkpoints.java @@ -38,6 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.DataInputStream; @@ -292,6 +293,13 @@ public static void disposeSavepoint( checkNotNull(configuration, "configuration"); checkNotNull(classLoader, "classLoader"); + StateBackend backend = loadStateBackend(configuration, classLoader, logger); + + disposeSavepoint(pointer, backend, classLoader); + } + + @Nonnull + public static StateBackend loadStateBackend(Configuration configuration, ClassLoader classLoader, @Nullable Logger logger) { if (logger != null) { logger.info("Attempting to load configured state backend for savepoint disposal"); } @@ -318,8 +326,7 @@ public static void disposeSavepoint( // FileSystem-based for metadata backend = new MemoryStateBackend(); } - - disposeSavepoint(pointer, backend, classLoader); + return backend; } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 68b40468a29b8..008d4dcdfe954 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -24,6 +24,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.checkpoint.Checkpoints; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -75,6 +76,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.stream.Collectors; /** @@ -319,6 +321,25 @@ public CompletableFuture> listJobs(Time timeout) { Collections.unmodifiableSet(new HashSet<>(jobManagerRunners.keySet()))); } + @Override + public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) { + final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + + return CompletableFuture.supplyAsync( + () -> { + log.info("Disposing savepoint {}.", savepointPath); + + try { + Checkpoints.disposeSavepoint(savepointPath, configuration, classLoader, log); + } catch (IOException | FlinkException e) { + throw new CompletionException(new FlinkException(String.format("Could not dispose savepoint %s.", savepointPath), e)); + } + + return Acknowledge.get(); + }, + jobManagerSharedServices.getScheduledExecutorService()); + } + @Override public CompletableFuture cancelJob(JobID jobId, Time timeout) { JobManagerRunner jobManagerRunner = jobManagerRunners.get(jobId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 0da6f333b6830..66770c52bdd77 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -534,6 +534,18 @@ public CompletableFuture triggerSavepoint(JobID jobId, String targetDire } } + public CompletableFuture disposeSavepoint(String savepointPath) { + try { + return getDispatcherGateway().disposeSavepoint(savepointPath, rpcTimeout); + } catch (LeaderRetrievalException | InterruptedException e) { + ExceptionUtils.checkInterrupted(e); + return FutureUtils.completedExceptionally( + new FlinkException( + String.format("Could not dispose savepoint %s.", savepointPath), + e)); + } + } + public CompletableFuture getExecutionGraph(JobID jobId) { try { return getDispatcherGateway().requestJob(jobId, rpcTimeout); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/OperationKey.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/OperationKey.java index a601e56a8831b..2f6e4bd851319 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/OperationKey.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/async/OperationKey.java @@ -27,11 +27,11 @@ * Any operation key for the {@link AbstractAsynchronousOperationHandlers} must extend this class. * It is used to store the trigger id. */ -public abstract class OperationKey { +public class OperationKey { private final TriggerId triggerId; - protected OperationKey(TriggerId triggerId) { + public OperationKey(TriggerId triggerId) { this.triggerId = Preconditions.checkNotNull(triggerId); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointDisposalHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointDisposalHandlers.java new file mode 100644 index 0000000000000..3cf5f59ca694b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/savepoints/SavepointDisposalHandlers.java @@ -0,0 +1,112 @@ +/* + * 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.rest.handler.job.savepoints; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rest.handler.HandlerRequest; +import org.apache.flink.runtime.rest.handler.async.AbstractAsynchronousOperationHandlers; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationInfo; +import org.apache.flink.runtime.rest.handler.async.OperationKey; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.TriggerId; +import org.apache.flink.runtime.rest.messages.TriggerIdPathParameter; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalRequest; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusHeaders; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusMessageParameters; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalTriggerHeaders; +import org.apache.flink.runtime.rpc.RpcUtils; +import org.apache.flink.runtime.webmonitor.RestfulGateway; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.SerializedThrowable; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** + * Handlers to trigger the disposal of a savepoint. + */ +public class SavepointDisposalHandlers extends AbstractAsynchronousOperationHandlers { + + /** + * {@link TriggerHandler} implementation for the savepoint disposal operation. + */ + public class SavepointDisposalTriggerHandler extends TriggerHandler { + + public SavepointDisposalTriggerHandler( + CompletableFuture localRestAddress, + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders) { + super( + localRestAddress, + leaderRetriever, + timeout, + responseHeaders, + SavepointDisposalTriggerHeaders.getInstance()); + } + + @Override + protected CompletableFuture triggerOperation(HandlerRequest request, RestfulGateway gateway) { + final String savepointPath = request.getRequestBody().getSavepointPath(); + return gateway.disposeSavepoint(savepointPath, RpcUtils.INF_TIMEOUT); + } + + @Override + protected OperationKey createOperationKey(HandlerRequest request) { + return new OperationKey(new TriggerId()); + } + } + + /** + * {@link StatusHandler} implementation for the savepoint disposal operation. + */ + public class SavepointDisposalStatusHandler extends StatusHandler { + + public SavepointDisposalStatusHandler( + CompletableFuture localRestAddress, + GatewayRetriever leaderRetriever, + Time timeout, + Map responseHeaders) { + super( + localRestAddress, + leaderRetriever, + timeout, + responseHeaders, + SavepointDisposalStatusHeaders.getInstance()); + } + + @Override + protected OperationKey getOperationKey(HandlerRequest request) { + final TriggerId triggerId = request.getPathParameter(TriggerIdPathParameter.class); + return new OperationKey(triggerId); + } + + @Override + protected AsynchronousOperationInfo exceptionalOperationResultResponse(Throwable throwable) { + return AsynchronousOperationInfo.completeExceptional(new SerializedThrowable(throwable)); + } + + @Override + protected AsynchronousOperationInfo operationResultResponse(Acknowledge operationResult) { + return AsynchronousOperationInfo.complete(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequest.java new file mode 100644 index 0000000000000..229ae9107aba8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequest.java @@ -0,0 +1,49 @@ +/* + * 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.rest.messages.job.savepoints; + +import org.apache.flink.runtime.rest.messages.RequestBody; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nonnull; + +/** + * Request body for a savepoint disposal call. + */ +public class SavepointDisposalRequest implements RequestBody { + + private static final String FIELD_NAME_SAVEPOINT_PATH = "savepoint-path"; + + @JsonProperty(FIELD_NAME_SAVEPOINT_PATH) + private final String savepointPath; + + @JsonCreator + public SavepointDisposalRequest( + @JsonProperty(FIELD_NAME_SAVEPOINT_PATH) @Nonnull String savepointPath) { + this.savepointPath = savepointPath; + } + + @JsonIgnore + public String getSavepointPath() { + return savepointPath; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java new file mode 100644 index 0000000000000..74deffdc492b0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusHeaders.java @@ -0,0 +1,75 @@ +/* + * 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.rest.messages.job.savepoints; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationInfo; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationStatusMessageHeaders; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationTriggerMessageHeaders; +import org.apache.flink.runtime.rest.handler.job.savepoints.SavepointDisposalHandlers; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.TriggerIdPathParameter; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * {@link AsynchronousOperationTriggerMessageHeaders} implementation for the {@link SavepointDisposalHandlers.SavepointDisposalStatusHandler}. + */ +public class SavepointDisposalStatusHeaders extends AsynchronousOperationStatusMessageHeaders { + + private static final SavepointDisposalStatusHeaders INSTANCE = new SavepointDisposalStatusHeaders(); + + private static final String URL = String.format("/savepoint-disposal/:%s", TriggerIdPathParameter.KEY); + + private SavepointDisposalStatusHeaders() {} + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public Class getRequestClass() { + return EmptyRequestBody.class; + } + + @Override + public SavepointDisposalStatusMessageParameters getUnresolvedMessageParameters() { + return new SavepointDisposalStatusMessageParameters(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.GET; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } + + public static SavepointDisposalStatusHeaders getInstance() { + return INSTANCE; + } + + @Override + protected Class getValueClass() { + return AsynchronousOperationInfo.class; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusMessageParameters.java new file mode 100644 index 0000000000000..d8804c9de6a6a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalStatusMessageParameters.java @@ -0,0 +1,46 @@ +/* + * 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.rest.messages.job.savepoints; + +import org.apache.flink.runtime.rest.handler.job.savepoints.SavepointDisposalHandlers; +import org.apache.flink.runtime.rest.messages.MessageParameters; +import org.apache.flink.runtime.rest.messages.MessagePathParameter; +import org.apache.flink.runtime.rest.messages.MessageQueryParameter; +import org.apache.flink.runtime.rest.messages.TriggerIdPathParameter; + +import java.util.Collection; +import java.util.Collections; + +/** + * {@link MessageParameters} for the {@link SavepointDisposalHandlers.SavepointDisposalStatusHandler}. + */ +public class SavepointDisposalStatusMessageParameters extends MessageParameters { + + public final TriggerIdPathParameter triggerIdPathParameter = new TriggerIdPathParameter(); + + @Override + public Collection> getPathParameters() { + return Collections.singleton(triggerIdPathParameter); + } + + @Override + public Collection> getQueryParameters() { + return Collections.emptyList(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalTriggerHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalTriggerHeaders.java new file mode 100644 index 0000000000000..5786498505a86 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalTriggerHeaders.java @@ -0,0 +1,67 @@ +/* + * 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.rest.messages.job.savepoints; + +import org.apache.flink.runtime.rest.HttpMethodWrapper; +import org.apache.flink.runtime.rest.handler.async.AsynchronousOperationTriggerMessageHeaders; +import org.apache.flink.runtime.rest.handler.job.savepoints.SavepointDisposalHandlers; +import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; + +/** + * {@link AsynchronousOperationTriggerMessageHeaders} for the {@link SavepointDisposalHandlers.SavepointDisposalTriggerHandler}. + */ +public class SavepointDisposalTriggerHeaders extends AsynchronousOperationTriggerMessageHeaders { + + private static final SavepointDisposalTriggerHeaders INSTANCE = new SavepointDisposalTriggerHeaders(); + + private static final String URL = "/savepoint-disposal"; + + private SavepointDisposalTriggerHeaders() {} + + @Override + public HttpResponseStatus getResponseStatusCode() { + return HttpResponseStatus.OK; + } + + @Override + public Class getRequestClass() { + return SavepointDisposalRequest.class; + } + + @Override + public EmptyMessageParameters getUnresolvedMessageParameters() { + return EmptyMessageParameters.getInstance(); + } + + @Override + public HttpMethodWrapper getHttpMethod() { + return HttpMethodWrapper.POST; + } + + @Override + public String getTargetRestEndpointURL() { + return URL; + } + + public static SavepointDisposalTriggerHeaders getInstance() { + return INSTANCE; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java index 471420604c259..6bb088c977580 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java @@ -144,6 +144,19 @@ default CompletableFuture triggerSavepoint( throw new UnsupportedOperationException(); } + /** + * Dispose the given savepoint. + * + * @param savepointPath identifying the savepoint to dispose + * @param timeout RPC timeout + * @return A future acknowledge if the disposal succeeded + */ + default CompletableFuture disposeSavepoint( + final String savepointPath, + @RpcTimeout final Time timeout) { + throw new UnsupportedOperationException(); + } + /** * Request the {@link JobStatus} of the given job. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java index d4aa94e19feaa..af346a7f3ddf4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java @@ -64,6 +64,7 @@ import org.apache.flink.runtime.rest.handler.job.rescaling.RescalingHandlers; import org.apache.flink.runtime.rest.handler.job.rescaling.RescalingStatusHeaders; import org.apache.flink.runtime.rest.handler.job.rescaling.RescalingTriggerHeaders; +import org.apache.flink.runtime.rest.handler.job.savepoints.SavepointDisposalHandlers; import org.apache.flink.runtime.rest.handler.job.savepoints.SavepointHandlers; import org.apache.flink.runtime.rest.handler.legacy.ConstantTextHandler; import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache; @@ -108,6 +109,8 @@ import org.apache.flink.runtime.rest.messages.job.metrics.JobVertexMetricsHeaders; import org.apache.flink.runtime.rest.messages.job.metrics.SubtaskMetricsHeaders; import org.apache.flink.runtime.rest.messages.job.metrics.TaskManagerMetricsHeaders; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalStatusHeaders; +import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointDisposalTriggerHeaders; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointStatusHeaders; import org.apache.flink.runtime.rest.messages.job.savepoints.SavepointTriggerHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerDetailsHeaders; @@ -499,6 +502,20 @@ protected List> initiali executor, metricFetcher); + final SavepointDisposalHandlers savepointDisposalHandlers = new SavepointDisposalHandlers(); + + final SavepointDisposalHandlers.SavepointDisposalTriggerHandler savepointDisposalTriggerHandler = savepointDisposalHandlers.new SavepointDisposalTriggerHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders); + + final SavepointDisposalHandlers.SavepointDisposalStatusHandler savepointDisposalStatusHandler = savepointDisposalHandlers.new SavepointDisposalStatusHandler( + restAddressFuture, + leaderRetriever, + timeout, + responseHeaders); + final Path webUiDir = restConfiguration.getWebUiDir(); Optional> optWebContent; @@ -549,6 +566,8 @@ protected List> initiali handlers.add(Tuple2.of(JobVertexDetailsHeaders.getInstance(), jobVertexDetailsHandler)); handlers.add(Tuple2.of(RescalingTriggerHeaders.getInstance(), rescalingTriggerHandler)); handlers.add(Tuple2.of(RescalingStatusHeaders.getInstance(), rescalingStatusHandler)); + handlers.add(Tuple2.of(SavepointDisposalTriggerHeaders.getInstance(), savepointDisposalTriggerHandler)); + handlers.add(Tuple2.of(SavepointDisposalStatusHeaders.getInstance(), savepointDisposalStatusHandler)); // TODO: Remove once the Yarn proxy can forward all REST verbs handlers.add(Tuple2.of(YarnCancelJobTerminationHeaders.getInstance(), jobCancelTerminationHandler)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 71c391f20a7c3..8ea686bfe530c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -25,7 +25,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.VoidBlobStore; +import org.apache.flink.runtime.checkpoint.Checkpoints; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.savepoint.SavepointV2; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.executiongraph.ErrorInfo; @@ -54,6 +56,11 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.state.CheckpointMetadataOutputStream; +import org.apache.flink.runtime.state.CheckpointStorage; +import org.apache.flink.runtime.state.CheckpointStorageLocation; +import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; +import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.InMemorySubmittedJobGraphStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; @@ -72,9 +79,18 @@ import org.junit.rules.TestName; import org.mockito.Mockito; +import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Collection; +import java.util.Collections; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -126,6 +142,8 @@ public class DispatcherTest extends TestLogger { private RunningJobsRegistry runningJobsRegistry; + private Configuration configuration; + /** Instance under test. */ private TestingDispatcher dispatcher; @@ -165,18 +183,19 @@ public void setUp() throws Exception { haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); runningJobsRegistry = haServices.getRunningJobsRegistry(); - final Configuration blobServerConfig = new Configuration(); - blobServerConfig.setString( + configuration = new Configuration(); + + configuration.setString( BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); dispatcher = new TestingDispatcher( rpcService, Dispatcher.DISPATCHER_NAME + '_' + name.getMethodName(), - new Configuration(), + configuration, haServices, mock(ResourceManagerGateway.class), - new BlobServer(blobServerConfig, new VoidBlobStore()), + new BlobServer(configuration, new VoidBlobStore()), heartbeatServices, UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), null, @@ -344,6 +363,42 @@ public void testJobRecovery() throws Exception { assertThat(jobIds, contains(jobGraph.getJobID())); } + /** + * Tests that we can dispose a savepoint. + */ + @Test + public void testSavepointDisposal() throws Exception { + final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); + + dispatcherLeaderElectionService.isLeader(UUID.randomUUID()).get(); + + final URI externalPointer = createTestingSavepoint(); + final Path savepointPath = Paths.get(externalPointer); + + assertThat(Files.exists(savepointPath), is(true)); + + dispatcherGateway.disposeSavepoint(externalPointer.toString(), TIMEOUT).get(); + + assertThat(Files.exists(savepointPath), is(false)); + } + + @Nonnull + private URI createTestingSavepoint() throws IOException, URISyntaxException { + final StateBackend stateBackend = Checkpoints.loadStateBackend(configuration, Thread.currentThread().getContextClassLoader(), log); + final CheckpointStorage checkpointStorage = stateBackend.createCheckpointStorage(jobGraph.getJobID()); + final File savepointFile = temporaryFolder.newFolder(); + final long checkpointId = 1L; + + final CheckpointStorageLocation checkpointStorageLocation = checkpointStorage.initializeLocationForSavepoint(checkpointId, savepointFile.getAbsolutePath()); + + final CheckpointMetadataOutputStream metadataOutputStream = checkpointStorageLocation.createMetadataOutputStream(); + Checkpoints.storeCheckpointMetadata(new SavepointV2(checkpointId, Collections.emptyList(), Collections.emptyList()), metadataOutputStream); + + final CompletedCheckpointStorageLocation completedCheckpointStorageLocation = metadataOutputStream.closeAndFinalizeCheckpoint(); + + return new URI(completedCheckpointStorageLocation.getExternalPointer()); + } + private static class TestingDispatcher extends Dispatcher { private final CountDownLatch submitJobLatch = new CountDownLatch(2); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequestTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequestTest.java new file mode 100644 index 0000000000000..3d5a90a6b9dd2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/savepoints/SavepointDisposalRequestTest.java @@ -0,0 +1,47 @@ +/* + * 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.rest.messages.job.savepoints; + +import org.apache.flink.runtime.rest.messages.RestRequestMarshallingTestBase; + +import org.hamcrest.Matchers; +import org.junit.Assert; + +import java.util.UUID; + +/** + * Tests the un/marshalling of the {@link SavepointDisposalRequest}. + */ +public class SavepointDisposalRequestTest extends RestRequestMarshallingTestBase { + + @Override + protected Class getTestRequestClass() { + return SavepointDisposalRequest.class; + } + + @Override + protected SavepointDisposalRequest getTestRequestInstance() { + return new SavepointDisposalRequest(UUID.randomUUID().toString()); + } + + @Override + protected void assertOriginalEqualsToUnmarshalled(SavepointDisposalRequest expected, SavepointDisposalRequest actual) { + Assert.assertThat(actual.getSavepointPath(), Matchers.is(Matchers.equalTo(expected.getSavepointPath()))); + } +} From 4ced9031e3700146e1b1af070390fcbc728cc624 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 26 Mar 2018 11:25:57 +0200 Subject: [PATCH 225/268] [hotfix] Improve logging in AkkaRpcActors --- .../java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java | 4 ++-- .../apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java | 5 ++++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java index a7d15d6e46c12..9f68ede224fc6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java @@ -140,8 +140,8 @@ public void onReceive(final Object message) { rpcEndpoint.getClass().getName(), message.getClass().getName()); - sendErrorIfSender(new AkkaRpcException("Discard message, because " + - "the rpc endpoint has not been started yet.")); + sendErrorIfSender(new AkkaRpcException( + String.format("Discard message, because the rpc endpoint %s has not been started yet.", rpcEndpoint.getAddress()))); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java index 57280fdd8b09f..6096439dbf6f0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java @@ -55,7 +55,10 @@ protected void handleMessage(Object message) { sendErrorIfSender( new FencingTokenException( - "Fencing token not set: Ignoring message " + message + " because the fencing token is null.")); + String.format( + "Fencing token not set: Ignoring message %s sent to %s because the fencing token is null.", + message, + rpcEndpoint.getAddress()))); } else { @SuppressWarnings("unchecked") FencedMessage fencedMessage = ((FencedMessage) message); From 106c6f60340d1659568f03fdd9559628f6ffff17 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 27 Mar 2018 08:31:56 +0200 Subject: [PATCH 226/268] [hotfix] Poll invariant variables out of polling loop in RestClusterClient#rescaleJob --- .../program/rest/RestClusterClient.java | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java index 912971456f668..cf683742240d4 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java @@ -507,21 +507,18 @@ public CompletableFuture rescaleJob(JobID jobId, int newParallelism final CompletableFuture rescalingOperationFuture = rescalingTriggerResponseFuture.thenCompose( (TriggerResponse triggerResponse) -> { final TriggerId triggerId = triggerResponse.getTriggerId(); + final RescalingStatusHeaders rescalingStatusHeaders = RescalingStatusHeaders.getInstance(); + final RescalingStatusMessageParameters rescalingStatusMessageParameters = rescalingStatusHeaders.getUnresolvedMessageParameters(); + + rescalingStatusMessageParameters.jobPathParameter.resolve(jobId); + rescalingStatusMessageParameters.triggerIdPathParameter.resolve(triggerId); return pollResourceAsync( - () -> { - final RescalingStatusHeaders rescalingStatusHeaders = RescalingStatusHeaders.getInstance(); - final RescalingStatusMessageParameters rescalingStatusMessageParameters = rescalingStatusHeaders.getUnresolvedMessageParameters(); - - rescalingStatusMessageParameters.jobPathParameter.resolve(jobId); - rescalingStatusMessageParameters.triggerIdPathParameter.resolve(triggerId); - return sendRetryableRequest( - rescalingStatusHeaders, - rescalingStatusMessageParameters, - EmptyRequestBody.getInstance(), - isConnectionProblemException()); - } - ); + () -> sendRetryableRequest( + rescalingStatusHeaders, + rescalingStatusMessageParameters, + EmptyRequestBody.getInstance(), + isConnectionProblemException())); }); return rescalingOperationFuture.thenApply( @@ -535,7 +532,7 @@ public CompletableFuture rescaleJob(JobID jobId, int newParallelism } @Override - public CompletableFuture disposeSavepoint(String savepointPath, Time timeout) { + public CompletableFuture disposeSavepoint(String savepointPath) { final SavepointDisposalRequest savepointDisposalRequest = new SavepointDisposalRequest(savepointPath); final CompletableFuture savepointDisposalTriggerFuture = sendRequest( From 26e0c02b2e877c1fa06aa8e0840137b53dbcd22b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 26 Mar 2018 13:46:56 +0200 Subject: [PATCH 227/268] [hotfix] Add Assert.fail to RestClusterClientTest --- .../apache/flink/client/program/rest/RestClusterClientTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index 926da924c5cd3..4202de292f0ea 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -472,6 +472,7 @@ testSavepointHandlers.new TestSavepointHandler( try { restClusterClient.triggerSavepoint(new JobID(), null).get(); + fail("Expected exception not thrown."); } catch (final ExecutionException e) { assertTrue( "RestClientException not in causal chain", From 871dccbdf9c1cdcba04e903a195eb9923bc56277 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 26 Mar 2018 15:16:09 +0200 Subject: [PATCH 228/268] [hotfix] Set RescalingHandlers timeout to RpcUtils.INF_TIMEOUT --- .../runtime/rest/handler/job/rescaling/RescalingHandlers.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingHandlers.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingHandlers.java index 3e4ae5a4e35e4..0efdd280766ad 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingHandlers.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescaling/RescalingHandlers.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.rest.messages.RescalingParallelismQueryParameter; import org.apache.flink.runtime.rest.messages.TriggerId; import org.apache.flink.runtime.rest.messages.TriggerIdPathParameter; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.util.SerializedThrowable; @@ -80,7 +81,7 @@ protected CompletableFuture triggerOperation(HandlerRequest Date: Fri, 23 Mar 2018 21:42:53 +0100 Subject: [PATCH 229/268] [FLINK-9067] [e2eTests] Add StreamSQLTestProgram and test run script. --- .../flink-stream-sql-test/pom.xml | 126 +++++++ .../flink/sql/tests/StreamSQLTestProgram.java | 309 ++++++++++++++++++ flink-end-to-end-tests/pom.xml | 1 + flink-end-to-end-tests/run-nightly-tests.sh | 8 + .../test-scripts/test_streaming_sql.sh | 43 +++ 5 files changed, 487 insertions(+) create mode 100644 flink-end-to-end-tests/flink-stream-sql-test/pom.xml create mode 100644 flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java create mode 100755 flink-end-to-end-tests/test-scripts/test_streaming_sql.sh diff --git a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml new file mode 100644 index 0000000000000..860432ce2daac --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml @@ -0,0 +1,126 @@ + + + + + + flink-end-to-end-tests + org.apache.flink + 1.6-SNAPSHOT + .. + + + 4.0.0 + + flink-stream-sql-test_${scala.binary.version} + flink-stream-sql-test + jar + + + + org.apache.flink + flink-core + ${project.version} + provided + + + org.apache.flink + flink-streaming-scala_${scala.binary.version} + ${project.version} + provided + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + provided + + + org.apache.flink + flink-connector-filesystem_${scala.binary.version} + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + package + + shade + + + + + com.google.code.findbugs:jsr305 + org.slf4j:* + log4j:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.apache.flink.sql.tests.StreamSQLTestProgram + + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + 1.7 + + + rename + package + + run + + + + + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java b/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java new file mode 100644 index 0000000000000..91fb4c11acf6e --- /dev/null +++ b/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java @@ -0,0 +1,309 @@ +/* + * 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.sql.tests; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.fs.bucketing.BasePathBucketer; +import org.apache.flink.streaming.connectors.fs.bucketing.BucketingSink; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.sources.DefinedFieldMapping; +import org.apache.flink.table.sources.DefinedRowtimeAttributes; +import org.apache.flink.table.sources.RowtimeAttributeDescriptor; +import org.apache.flink.table.sources.StreamTableSource; +import org.apache.flink.table.sources.tsextractors.ExistingField; +import org.apache.flink.table.sources.wmstrategies.BoundedOutOfOrderTimestamps; +import org.apache.flink.types.Row; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * End-to-end test for Stream SQL queries. + * + *

      Includes the following SQL features: + * - OVER window aggregation + * - keyed and non-keyed GROUP BY TUMBLE aggregation + * - windowed INNER JOIN + * - TableSource with event-time attribute + * + *

      The stream is bounded and will complete after about a minute. + * The result is always constant. + * The job is killed on the first attemped and restarted. + * + *

      Parameters: + * -outputPath Sets the path to where the result data is written. + */ +public class StreamSQLTestProgram { + + public static void main(String[] args) throws Exception { + + ParameterTool params = ParameterTool.fromArgs(args); + String outputPath = params.getRequired("outputPath"); + + StreamExecutionEnvironment sEnv = StreamExecutionEnvironment.getExecutionEnvironment(); + sEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); + sEnv.enableCheckpointing(4000); + sEnv.getConfig().setAutoWatermarkInterval(1000); + + StreamTableEnvironment tEnv = TableEnvironment.getTableEnvironment(sEnv); + + tEnv.registerTableSource("table1", new GeneratorTableSource(10, 100, 60, 0)); + tEnv.registerTableSource("table2", new GeneratorTableSource(5, 0.2f, 60, 5)); + + int overWindowSizeSeconds = 1; + int tumbleWindowSizeSeconds = 10; + + String overQuery = String.format( + "SELECT " + + " key, " + + " rowtime, " + + " COUNT(*) OVER (PARTITION BY key ORDER BY rowtime RANGE BETWEEN INTERVAL '%d' SECOND PRECEDING AND CURRENT ROW) AS cnt " + + "FROM table1", + overWindowSizeSeconds); + + String tumbleQuery = String.format( + "SELECT " + + " key, " + + " CASE SUM(cnt) / COUNT(*) WHEN 101 THEN 1 ELSE 99 END AS correct, " + + " TUMBLE_START(rowtime, INTERVAL '%d' SECOND) AS wStart, " + + " TUMBLE_ROWTIME(rowtime, INTERVAL '%d' SECOND) AS rowtime " + + "FROM (%s) " + + "WHERE rowtime > TIMESTAMP '1970-01-01 00:00:01' " + + "GROUP BY key, TUMBLE(rowtime, INTERVAL '%d' SECOND)", + tumbleWindowSizeSeconds, + tumbleWindowSizeSeconds, + overQuery, + tumbleWindowSizeSeconds); + + String joinQuery = String.format( + "SELECT " + + " t1.key, " + + " t2.rowtime AS rowtime, " + + " t2.correct," + + " t2.wStart " + + "FROM table2 t1, (%s) t2 " + + "WHERE " + + " t1.key = t2.key AND " + + " t1.rowtime BETWEEN t2.rowtime AND t2.rowtime + INTERVAL '%d' SECOND", + tumbleQuery, + tumbleWindowSizeSeconds); + + String finalAgg = String.format( + "SELECT " + + " SUM(correct) AS correct, " + + " TUMBLE_START(rowtime, INTERVAL '20' SECOND) AS rowtime " + + "FROM (%s) " + + "GROUP BY TUMBLE(rowtime, INTERVAL '20' SECOND)", + joinQuery); + + // get Table for SQL query + Table result = tEnv.sqlQuery(finalAgg); + // convert Table into append-only DataStream + DataStream resultStream = + tEnv.toAppendStream(result, Types.ROW(Types.INT, Types.SQL_TIMESTAMP)); + + // define bucketing sink to emit the result + BucketingSink sink = new BucketingSink(outputPath) + .setBucketer(new BasePathBucketer<>()); + + resultStream + // inject a KillMapper that forwards all records but terminates the first execution attempt + .map(new KillMapper()).setParallelism(1) + // add sink function + .addSink(sink).setParallelism(1); + + sEnv.execute(); + } + + /** + * TableSource for generated data. + */ + public static class GeneratorTableSource + implements StreamTableSource, DefinedRowtimeAttributes, DefinedFieldMapping { + + private final int numKeys; + private final float recordsPerKeyAndSecond; + private final int durationSeconds; + private final int offsetSeconds; + + public GeneratorTableSource(int numKeys, float recordsPerKeyAndSecond, int durationSeconds, int offsetSeconds) { + this.numKeys = numKeys; + this.recordsPerKeyAndSecond = recordsPerKeyAndSecond; + this.durationSeconds = durationSeconds; + this.offsetSeconds = offsetSeconds; + } + + @Override + public DataStream getDataStream(StreamExecutionEnvironment execEnv) { + return execEnv.addSource(new Generator(numKeys, recordsPerKeyAndSecond, durationSeconds, offsetSeconds)); + } + + @Override + public TypeInformation getReturnType() { + return Types.ROW(Types.INT, Types.LONG, Types.STRING); + } + + @Override + public TableSchema getTableSchema() { + return new TableSchema( + new String[] {"key", "rowtime", "payload"}, + new TypeInformation[] {Types.INT, Types.SQL_TIMESTAMP, Types.STRING}); + } + + @Override + public String explainSource() { + return "GeneratorTableSource"; + } + + @Override + public List getRowtimeAttributeDescriptors() { + return Collections.singletonList( + new RowtimeAttributeDescriptor( + "rowtime", + new ExistingField("ts"), + new BoundedOutOfOrderTimestamps(100))); + } + + @Override + public Map getFieldMapping() { + Map mapping = new HashMap<>(); + mapping.put("key", "f0"); + mapping.put("ts", "f1"); + mapping.put("payload", "f2"); + return mapping; + } + } + + /** + * Data-generating source function. + */ + public static class Generator implements SourceFunction, ResultTypeQueryable, ListCheckpointed { + + private final int numKeys; + private final int offsetSeconds; + + private final int sleepMs; + private final int durationMs; + + private long ms = 0; + + public Generator(int numKeys, float rowsPerKeyAndSecond, int durationSeconds, int offsetSeconds) { + this.numKeys = numKeys; + this.durationMs = durationSeconds * 1000; + this.offsetSeconds = offsetSeconds; + + this.sleepMs = (int) (1000 / rowsPerKeyAndSecond); + } + + @Override + public void run(SourceContext ctx) throws Exception { + long offsetMS = offsetSeconds * 2000L; + + while (ms < durationMs) { + synchronized (ctx.getCheckpointLock()) { + for (int i = 0; i < numKeys; i++) { + ctx.collect(Row.of(i, ms + offsetMS, "Some payload...")); + } + ms += sleepMs; + } + Thread.sleep(sleepMs); + } + } + + @Override + public void cancel() { } + + @Override + public TypeInformation getProducedType() { + return Types.ROW(Types.INT, Types.LONG, Types.STRING); + } + + @Override + public List snapshotState(long checkpointId, long timestamp) throws Exception { + return Collections.singletonList(ms); + } + + @Override + public void restoreState(List state) throws Exception { + for (Long l : state) { + ms += l; + } + } + } + + /** + * Kills the first execution attempt of an application when it receives the second record. + */ + public static class KillMapper implements MapFunction, ListCheckpointed, ResultTypeQueryable { + + // counts all processed records of all previous execution attempts + private int saveRecordCnt = 0; + // counts all processed records of this execution attempt + private int lostRecordCnt = 0; + + @Override + public Row map(Row value) throws Exception { + + // the both counts are the same only in the first execution attempt + if (saveRecordCnt == 1 && lostRecordCnt == 1) { + throw new RuntimeException("Kill this Job!"); + } + + // update checkpointed counter + saveRecordCnt++; + // update non-checkpointed counter + lostRecordCnt++; + + // forward record + return value; + } + + @Override + public TypeInformation getProducedType() { + return Types.ROW(Types.INT, Types.SQL_TIMESTAMP); + } + + @Override + public List snapshotState(long checkpointId, long timestamp) throws Exception { + return Collections.singletonList(saveRecordCnt); + } + + @Override + public void restoreState(List state) throws Exception { + for (Integer i : state) { + saveRecordCnt += i; + } + } + } + +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index db86cc1854b1c..26f192af38eda 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -37,6 +37,7 @@ under the License. flink-parent-child-classloading-test flink-dataset-allround-test + flink-stream-sql-test diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index 71224e09f1b7f..eca2e4db47af5 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -55,5 +55,13 @@ if [ $EXIT_CODE == 0 ]; then EXIT_CODE=$? fi +if [ $EXIT_CODE == 0 ]; then + printf "\n==============================================================================\n" + printf "Running Streaming SQL nightly end-to-end test\n" + printf "==============================================================================\n" + $END_TO_END_DIR/test-scripts/test_streaming_sql.sh + EXIT_CODE=$? +fi + # Exit code for Travis build success/failure exit $EXIT_CODE diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_sql.sh b/flink-end-to-end-tests/test-scripts/test_streaming_sql.sh new file mode 100755 index 0000000000000..ed7b16c852b60 --- /dev/null +++ b/flink-end-to-end-tests/test-scripts/test_streaming_sql.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash +################################################################################ +# 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. +################################################################################ + +source "$(dirname "$0")"/common.sh + +TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-stream-sql-test/target/StreamSQLTestProgram.jar + +# copy flink-table jar into lib folder +cp $FLINK_DIR/opt/flink-table*jar $FLINK_DIR/lib + +start_cluster +$FLINK_DIR/bin/taskmanager.sh start +$FLINK_DIR/bin/taskmanager.sh start +$FLINK_DIR/bin/taskmanager.sh start + +$FLINK_DIR/bin/flink run -p 4 $TEST_PROGRAM_JAR -outputPath $TEST_DATA_DIR/out/result + +stop_cluster +$FLINK_DIR/bin/taskmanager.sh stop-all + +# remove flink-table from lib folder +rm $FLINK_DIR/lib/flink-table*jar + +# collect results from files +cat /tmp/xxx/part-0-0 /tmp/xxx/_part-0-1.pending > $TEST_DATA_DIR/out/result-complete +# check result +check_result_hash "StreamSQL" $TEST_DATA_DIR/out/result-complete "b29f14ed221a936211202ff65b51ee26" From c5fa87f6923bb81320affd8a0c27250d8ad92b26 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 27 Mar 2018 14:20:47 +0200 Subject: [PATCH 230/268] [FLINK-9067] [e2eTests] Fix test and simplify code This closes #5759. --- .../flink-stream-sql-test/pom.xml | 21 +--------------- .../flink/sql/tests/StreamSQLTestProgram.java | 12 +++++----- .../test-scripts/test_streaming_sql.sh | 24 ++++++++++++++----- 3 files changed, 25 insertions(+), 32 deletions(-) diff --git a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml index 860432ce2daac..1b8e4f7a4d17b 100644 --- a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml +++ b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml @@ -73,6 +73,7 @@ shade + StreamSQLTestProgram com.google.code.findbugs:jsr305 @@ -99,26 +100,6 @@ - - - org.apache.maven.plugins - maven-antrun-plugin - 1.7 - - - rename - package - - run - - - - - - - - - diff --git a/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java b/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java index 91fb4c11acf6e..e9e79ac2745a9 100644 --- a/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java +++ b/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java @@ -58,7 +58,7 @@ * *

      The stream is bounded and will complete after about a minute. * The result is always constant. - * The job is killed on the first attemped and restarted. + * The job is killed on the first attempt and restarted. * *

      Parameters: * -outputPath Sets the path to where the result data is written. @@ -249,12 +249,12 @@ public TypeInformation getProducedType() { } @Override - public List snapshotState(long checkpointId, long timestamp) throws Exception { + public List snapshotState(long checkpointId, long timestamp) { return Collections.singletonList(ms); } @Override - public void restoreState(List state) throws Exception { + public void restoreState(List state) { for (Long l : state) { ms += l; } @@ -272,7 +272,7 @@ public static class KillMapper implements MapFunction, ListCheckpointe private int lostRecordCnt = 0; @Override - public Row map(Row value) throws Exception { + public Row map(Row value) { // the both counts are the same only in the first execution attempt if (saveRecordCnt == 1 && lostRecordCnt == 1) { @@ -294,12 +294,12 @@ public TypeInformation getProducedType() { } @Override - public List snapshotState(long checkpointId, long timestamp) throws Exception { + public List snapshotState(long checkpointId, long timestamp) { return Collections.singletonList(saveRecordCnt); } @Override - public void restoreState(List state) throws Exception { + public void restoreState(List state) { for (Integer i : state) { saveRecordCnt += i; } diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_sql.sh b/flink-end-to-end-tests/test-scripts/test_streaming_sql.sh index ed7b16c852b60..21c64a168c30e 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_sql.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_sql.sh @@ -31,13 +31,25 @@ $FLINK_DIR/bin/taskmanager.sh start $FLINK_DIR/bin/flink run -p 4 $TEST_PROGRAM_JAR -outputPath $TEST_DATA_DIR/out/result -stop_cluster -$FLINK_DIR/bin/taskmanager.sh stop-all +function sql_cleanup() { -# remove flink-table from lib folder -rm $FLINK_DIR/lib/flink-table*jar + stop_cluster + $FLINK_DIR/bin/taskmanager.sh stop-all + + # remove flink-table from lib folder + rm $FLINK_DIR/lib/flink-table*jar + + # make sure to run regular cleanup as well + cleanup +} +trap sql_cleanup INT +trap sql_cleanup EXIT # collect results from files -cat /tmp/xxx/part-0-0 /tmp/xxx/_part-0-1.pending > $TEST_DATA_DIR/out/result-complete -# check result +cat $TEST_DATA_DIR/out/result/part-0-0 $TEST_DATA_DIR/out/result/_part-0-1.pending > $TEST_DATA_DIR/out/result-complete + +# check result: +# 20,1970-01-01 00:00:00.0 +# 20,1970-01-01 00:00:20.0 +# 20,1970-01-01 00:00:40.0 check_result_hash "StreamSQL" $TEST_DATA_DIR/out/result-complete "b29f14ed221a936211202ff65b51ee26" From 8b4543bfd5bff2d64a0e4bf894b121e6acd3bd1c Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 27 Mar 2018 16:34:28 +0200 Subject: [PATCH 231/268] [hotfix] Fix wrong parent version --- flink-end-to-end-tests/flink-stream-sql-test/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml index 1b8e4f7a4d17b..2f345a59ea77e 100644 --- a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml +++ b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.6-SNAPSHOT + 1.5-SNAPSHOT .. From fb98e2a3b2cfbfe2de551a654e3e11c1052759d8 Mon Sep 17 00:00:00 2001 From: Matrix42 <934336389@qq.com> Date: Tue, 27 Mar 2018 14:28:07 +0800 Subject: [PATCH 232/268] [FLINK-9093] [docs] Ship jQuery library without external provider This closes #5770. --- docs/_layouts/base.html | 2 +- docs/page/js/jquery.min.js | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 docs/page/js/jquery.min.js diff --git a/docs/_layouts/base.html b/docs/_layouts/base.html index 1e360cf5dde1d..7d86d74f12b06 100644 --- a/docs/_layouts/base.html +++ b/docs/_layouts/base.html @@ -81,7 +81,7 @@ - + diff --git a/docs/page/js/jquery.min.js b/docs/page/js/jquery.min.js new file mode 100644 index 0000000000000..e6a051d0d1d32 --- /dev/null +++ b/docs/page/js/jquery.min.js @@ -0,0 +1,4 @@ +/*! jQuery v1.11.2 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k={},l="1.11.2",m=function(a,b){return new m.fn.init(a,b)},n=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,o=/^-ms-/,p=/-([\da-z])/gi,q=function(a,b){return b.toUpperCase()};m.fn=m.prototype={jquery:l,constructor:m,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=m.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return m.each(this,a,b)},map:function(a){return this.pushStack(m.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},m.extend=m.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||m.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(m.isPlainObject(c)||(b=m.isArray(c)))?(b?(b=!1,f=a&&m.isArray(a)?a:[]):f=a&&m.isPlainObject(a)?a:{},g[d]=m.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},m.extend({expando:"jQuery"+(l+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===m.type(a)},isArray:Array.isArray||function(a){return"array"===m.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){return!m.isArray(a)&&a-parseFloat(a)+1>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==m.type(a)||a.nodeType||m.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(k.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&m.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(o,"ms-").replace(p,q)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=r(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(n,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(r(Object(a))?m.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=r(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),m.isFunction(a)?(c=d.call(arguments,2),e=function(){return a.apply(b||this,c.concat(d.call(arguments)))},e.guid=a.guid=a.guid||m.guid++,e):void 0},now:function(){return+new Date},support:k}),m.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function r(a){var b=a.length,c=m.type(a);return"function"===c||m.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var s=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+1*new Date,v=a.document,w=0,x=0,y=hb(),z=hb(),A=hb(),B=function(a,b){return a===b&&(l=!0),0},C=1<<31,D={}.hasOwnProperty,E=[],F=E.pop,G=E.push,H=E.push,I=E.slice,J=function(a,b){for(var c=0,d=a.length;d>c;c++)if(a[c]===b)return c;return-1},K="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",L="[\\x20\\t\\r\\n\\f]",M="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",N=M.replace("w","w#"),O="\\["+L+"*("+M+")(?:"+L+"*([*^$|!~]?=)"+L+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+N+"))|)"+L+"*\\]",P=":("+M+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+O+")*)|.*)\\)|)",Q=new RegExp(L+"+","g"),R=new RegExp("^"+L+"+|((?:^|[^\\\\])(?:\\\\.)*)"+L+"+$","g"),S=new RegExp("^"+L+"*,"+L+"*"),T=new RegExp("^"+L+"*([>+~]|"+L+")"+L+"*"),U=new RegExp("="+L+"*([^\\]'\"]*?)"+L+"*\\]","g"),V=new RegExp(P),W=new RegExp("^"+N+"$"),X={ID:new RegExp("^#("+M+")"),CLASS:new RegExp("^\\.("+M+")"),TAG:new RegExp("^("+M.replace("w","w*")+")"),ATTR:new RegExp("^"+O),PSEUDO:new RegExp("^"+P),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+L+"*(even|odd|(([+-]|)(\\d*)n|)"+L+"*(?:([+-]|)"+L+"*(\\d+)|))"+L+"*\\)|)","i"),bool:new RegExp("^(?:"+K+")$","i"),needsContext:new RegExp("^"+L+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+L+"*((?:-\\d)?\\d*)"+L+"*\\)|)(?=[^-]|$)","i")},Y=/^(?:input|select|textarea|button)$/i,Z=/^h\d$/i,$=/^[^{]+\{\s*\[native \w/,_=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ab=/[+~]/,bb=/'|\\/g,cb=new RegExp("\\\\([\\da-f]{1,6}"+L+"?|("+L+")|.)","ig"),db=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)},eb=function(){m()};try{H.apply(E=I.call(v.childNodes),v.childNodes),E[v.childNodes.length].nodeType}catch(fb){H={apply:E.length?function(a,b){G.apply(a,I.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function gb(a,b,d,e){var f,h,j,k,l,o,r,s,w,x;if((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,d=d||[],k=b.nodeType,"string"!=typeof a||!a||1!==k&&9!==k&&11!==k)return d;if(!e&&p){if(11!==k&&(f=_.exec(a)))if(j=f[1]){if(9===k){if(h=b.getElementById(j),!h||!h.parentNode)return d;if(h.id===j)return d.push(h),d}else if(b.ownerDocument&&(h=b.ownerDocument.getElementById(j))&&t(b,h)&&h.id===j)return d.push(h),d}else{if(f[2])return H.apply(d,b.getElementsByTagName(a)),d;if((j=f[3])&&c.getElementsByClassName)return H.apply(d,b.getElementsByClassName(j)),d}if(c.qsa&&(!q||!q.test(a))){if(s=r=u,w=b,x=1!==k&&a,1===k&&"object"!==b.nodeName.toLowerCase()){o=g(a),(r=b.getAttribute("id"))?s=r.replace(bb,"\\$&"):b.setAttribute("id",s),s="[id='"+s+"'] ",l=o.length;while(l--)o[l]=s+rb(o[l]);w=ab.test(a)&&pb(b.parentNode)||b,x=o.join(",")}if(x)try{return H.apply(d,w.querySelectorAll(x)),d}catch(y){}finally{r||b.removeAttribute("id")}}}return i(a.replace(R,"$1"),b,d,e)}function hb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function ib(a){return a[u]=!0,a}function jb(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function kb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function lb(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||C)-(~a.sourceIndex||C);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function mb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function nb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function ob(a){return ib(function(b){return b=+b,ib(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function pb(a){return a&&"undefined"!=typeof a.getElementsByTagName&&a}c=gb.support={},f=gb.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=gb.setDocument=function(a){var b,e,g=a?a.ownerDocument||a:v;return g!==n&&9===g.nodeType&&g.documentElement?(n=g,o=g.documentElement,e=g.defaultView,e&&e!==e.top&&(e.addEventListener?e.addEventListener("unload",eb,!1):e.attachEvent&&e.attachEvent("onunload",eb)),p=!f(g),c.attributes=jb(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=jb(function(a){return a.appendChild(g.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=$.test(g.getElementsByClassName),c.getById=jb(function(a){return o.appendChild(a).id=u,!g.getElementsByName||!g.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){var c="undefined"!=typeof a.getAttributeNode&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return"undefined"!=typeof b.getElementsByTagName?b.getElementsByTagName(a):c.qsa?b.querySelectorAll(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=$.test(g.querySelectorAll))&&(jb(function(a){o.appendChild(a).innerHTML="",a.querySelectorAll("[msallowcapture^='']").length&&q.push("[*^$]="+L+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+L+"*(?:value|"+K+")"),a.querySelectorAll("[id~="+u+"-]").length||q.push("~="),a.querySelectorAll(":checked").length||q.push(":checked"),a.querySelectorAll("a#"+u+"+*").length||q.push(".#.+[+~]")}),jb(function(a){var b=g.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+L+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=$.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&jb(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",P)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=$.test(o.compareDocumentPosition),t=b||$.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===g||a.ownerDocument===v&&t(v,a)?-1:b===g||b.ownerDocument===v&&t(v,b)?1:k?J(k,a)-J(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,e=a.parentNode,f=b.parentNode,h=[a],i=[b];if(!e||!f)return a===g?-1:b===g?1:e?-1:f?1:k?J(k,a)-J(k,b):0;if(e===f)return lb(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)i.unshift(c);while(h[d]===i[d])d++;return d?lb(h[d],i[d]):h[d]===v?-1:i[d]===v?1:0},g):n},gb.matches=function(a,b){return gb(a,null,null,b)},gb.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(U,"='$1']"),!(!c.matchesSelector||!p||r&&r.test(b)||q&&q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return gb(b,n,null,[a]).length>0},gb.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},gb.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&D.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},gb.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},gb.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=gb.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=gb.selectors={cacheLength:50,createPseudo:ib,match:X,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(cb,db),a[3]=(a[3]||a[4]||a[5]||"").replace(cb,db),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||gb.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&gb.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return X.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&V.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(cb,db).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+L+")"+a+"("+L+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||"undefined"!=typeof a.getAttribute&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=gb.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e.replace(Q," ")+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){k=q[u]||(q[u]={}),j=k[a]||[],n=j[0]===w&&j[1],m=j[0]===w&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[w,n,m];break}}else if(s&&(j=(b[u]||(b[u]={}))[a])&&j[0]===w)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(s&&((l[u]||(l[u]={}))[a]=[w,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||gb.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?ib(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=J(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:ib(function(a){var b=[],c=[],d=h(a.replace(R,"$1"));return d[u]?ib(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),b[0]=null,!c.pop()}}),has:ib(function(a){return function(b){return gb(a,b).length>0}}),contains:ib(function(a){return a=a.replace(cb,db),function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:ib(function(a){return W.test(a||"")||gb.error("unsupported lang: "+a),a=a.replace(cb,db).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Z.test(a.nodeName)},input:function(a){return Y.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:ob(function(){return[0]}),last:ob(function(a,b){return[b-1]}),eq:ob(function(a,b,c){return[0>c?c+b:c]}),even:ob(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:ob(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:ob(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:ob(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function sb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(i=b[u]||(b[u]={}),(h=i[d])&&h[0]===w&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function tb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function ub(a,b,c){for(var d=0,e=b.length;e>d;d++)gb(a,b[d],c);return c}function vb(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function wb(a,b,c,d,e,f){return d&&!d[u]&&(d=wb(d)),e&&!e[u]&&(e=wb(e,f)),ib(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||ub(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:vb(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=vb(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?J(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=vb(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):H.apply(g,r)})}function xb(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=sb(function(a){return a===b},h,!0),l=sb(function(a){return J(b,a)>-1},h,!0),m=[function(a,c,d){var e=!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d));return b=null,e}];f>i;i++)if(c=d.relative[a[i].type])m=[sb(tb(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return wb(i>1&&tb(m),i>1&&rb(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(R,"$1"),c,e>i&&xb(a.slice(i,e)),f>e&&xb(a=a.slice(e)),f>e&&rb(a))}m.push(c)}return tb(m)}function yb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,m,o,p=0,q="0",r=f&&[],s=[],t=j,u=f||e&&d.find.TAG("*",k),v=w+=null==t?1:Math.random()||.1,x=u.length;for(k&&(j=g!==n&&g);q!==x&&null!=(l=u[q]);q++){if(e&&l){m=0;while(o=a[m++])if(o(l,g,h)){i.push(l);break}k&&(w=v)}c&&((l=!o&&l)&&p--,f&&r.push(l))}if(p+=q,c&&q!==p){m=0;while(o=b[m++])o(r,s,g,h);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=F.call(i));s=vb(s)}H.apply(i,s),k&&!f&&s.length>0&&p+b.length>1&&gb.uniqueSort(i)}return k&&(w=v,j=t),r};return c?ib(f):f}return h=gb.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=xb(b[c]),f[u]?d.push(f):e.push(f);f=A(a,yb(e,d)),f.selector=a}return f},i=gb.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(cb,db),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=X.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(cb,db),ab.test(j[0].type)&&pb(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&rb(j),!a)return H.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,ab.test(a)&&pb(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=jb(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),jb(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||kb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&jb(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||kb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),jb(function(a){return null==a.getAttribute("disabled")})||kb(K,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),gb}(a);m.find=s,m.expr=s.selectors,m.expr[":"]=m.expr.pseudos,m.unique=s.uniqueSort,m.text=s.getText,m.isXMLDoc=s.isXML,m.contains=s.contains;var t=m.expr.match.needsContext,u=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,v=/^.[^:#\[\.,]*$/;function w(a,b,c){if(m.isFunction(b))return m.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return m.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(v.test(b))return m.filter(b,a,c);b=m.filter(b,a)}return m.grep(a,function(a){return m.inArray(a,b)>=0!==c})}m.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?m.find.matchesSelector(d,a)?[d]:[]:m.find.matches(a,m.grep(b,function(a){return 1===a.nodeType}))},m.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(m(a).filter(function(){for(b=0;e>b;b++)if(m.contains(d[b],this))return!0}));for(b=0;e>b;b++)m.find(a,d[b],c);return c=this.pushStack(e>1?m.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(w(this,a||[],!1))},not:function(a){return this.pushStack(w(this,a||[],!0))},is:function(a){return!!w(this,"string"==typeof a&&t.test(a)?m(a):a||[],!1).length}});var x,y=a.document,z=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,A=m.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:z.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||x).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof m?b[0]:b,m.merge(this,m.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:y,!0)),u.test(c[1])&&m.isPlainObject(b))for(c in b)m.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=y.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return x.find(a);this.length=1,this[0]=d}return this.context=y,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):m.isFunction(a)?"undefined"!=typeof x.ready?x.ready(a):a(m):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),m.makeArray(a,this))};A.prototype=m.fn,x=m(y);var B=/^(?:parents|prev(?:Until|All))/,C={children:!0,contents:!0,next:!0,prev:!0};m.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!m(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),m.fn.extend({has:function(a){var b,c=m(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(m.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=t.test(a)||"string"!=typeof a?m(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&m.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?m.unique(f):f)},index:function(a){return a?"string"==typeof a?m.inArray(this[0],m(a)):m.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(m.unique(m.merge(this.get(),m(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function D(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}m.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return m.dir(a,"parentNode")},parentsUntil:function(a,b,c){return m.dir(a,"parentNode",c)},next:function(a){return D(a,"nextSibling")},prev:function(a){return D(a,"previousSibling")},nextAll:function(a){return m.dir(a,"nextSibling")},prevAll:function(a){return m.dir(a,"previousSibling")},nextUntil:function(a,b,c){return m.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return m.dir(a,"previousSibling",c)},siblings:function(a){return m.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return m.sibling(a.firstChild)},contents:function(a){return m.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:m.merge([],a.childNodes)}},function(a,b){m.fn[a]=function(c,d){var e=m.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=m.filter(d,e)),this.length>1&&(C[a]||(e=m.unique(e)),B.test(a)&&(e=e.reverse())),this.pushStack(e)}});var E=/\S+/g,F={};function G(a){var b=F[a]={};return m.each(a.match(E)||[],function(a,c){b[c]=!0}),b}m.Callbacks=function(a){a="string"==typeof a?F[a]||G(a):m.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){m.each(b,function(b,c){var d=m.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&m.each(arguments,function(a,c){var d;while((d=m.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?m.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c.slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},m.extend({Deferred:function(a){var b=[["resolve","done",m.Callbacks("once memory"),"resolved"],["reject","fail",m.Callbacks("once memory"),"rejected"],["notify","progress",m.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return m.Deferred(function(c){m.each(b,function(b,f){var g=m.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&m.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?m.extend(a,d):d}},e={};return d.pipe=d.then,m.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&m.isFunction(a.promise)?e:0,g=1===f?a:m.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&m.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var H;m.fn.ready=function(a){return m.ready.promise().done(a),this},m.extend({isReady:!1,readyWait:1,holdReady:function(a){a?m.readyWait++:m.ready(!0)},ready:function(a){if(a===!0?!--m.readyWait:!m.isReady){if(!y.body)return setTimeout(m.ready);m.isReady=!0,a!==!0&&--m.readyWait>0||(H.resolveWith(y,[m]),m.fn.triggerHandler&&(m(y).triggerHandler("ready"),m(y).off("ready")))}}});function I(){y.addEventListener?(y.removeEventListener("DOMContentLoaded",J,!1),a.removeEventListener("load",J,!1)):(y.detachEvent("onreadystatechange",J),a.detachEvent("onload",J))}function J(){(y.addEventListener||"load"===event.type||"complete"===y.readyState)&&(I(),m.ready())}m.ready.promise=function(b){if(!H)if(H=m.Deferred(),"complete"===y.readyState)setTimeout(m.ready);else if(y.addEventListener)y.addEventListener("DOMContentLoaded",J,!1),a.addEventListener("load",J,!1);else{y.attachEvent("onreadystatechange",J),a.attachEvent("onload",J);var c=!1;try{c=null==a.frameElement&&y.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!m.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}I(),m.ready()}}()}return H.promise(b)};var K="undefined",L;for(L in m(k))break;k.ownLast="0"!==L,k.inlineBlockNeedsLayout=!1,m(function(){var a,b,c,d;c=y.getElementsByTagName("body")[0],c&&c.style&&(b=y.createElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),typeof b.style.zoom!==K&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",k.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(d))}),function(){var a=y.createElement("div");if(null==k.deleteExpando){k.deleteExpando=!0;try{delete a.test}catch(b){k.deleteExpando=!1}}a=null}(),m.acceptData=function(a){var b=m.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var M=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,N=/([A-Z])/g;function O(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(N,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:M.test(c)?m.parseJSON(c):c}catch(e){}m.data(a,b,c)}else c=void 0}return c}function P(a){var b;for(b in a)if(("data"!==b||!m.isEmptyObject(a[b]))&&"toJSON"!==b)return!1; +return!0}function Q(a,b,d,e){if(m.acceptData(a)){var f,g,h=m.expando,i=a.nodeType,j=i?m.cache:a,k=i?a[h]:a[h]&&h;if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||m.guid++:h),j[k]||(j[k]=i?{}:{toJSON:m.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=m.extend(j[k],b):j[k].data=m.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[m.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[m.camelCase(b)])):f=g,f}}function R(a,b,c){if(m.acceptData(a)){var d,e,f=a.nodeType,g=f?m.cache:a,h=f?a[m.expando]:m.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){m.isArray(b)?b=b.concat(m.map(b,m.camelCase)):b in d?b=[b]:(b=m.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!P(d):!m.isEmptyObject(d))return}(c||(delete g[h].data,P(g[h])))&&(f?m.cleanData([a],!0):k.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}m.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?m.cache[a[m.expando]]:a[m.expando],!!a&&!P(a)},data:function(a,b,c){return Q(a,b,c)},removeData:function(a,b){return R(a,b)},_data:function(a,b,c){return Q(a,b,c,!0)},_removeData:function(a,b){return R(a,b,!0)}}),m.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=m.data(f),1===f.nodeType&&!m._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=m.camelCase(d.slice(5)),O(f,d,e[d])));m._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){m.data(this,a)}):arguments.length>1?this.each(function(){m.data(this,a,b)}):f?O(f,a,m.data(f,a)):void 0},removeData:function(a){return this.each(function(){m.removeData(this,a)})}}),m.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=m._data(a,b),c&&(!d||m.isArray(c)?d=m._data(a,b,m.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=m.queue(a,b),d=c.length,e=c.shift(),f=m._queueHooks(a,b),g=function(){m.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return m._data(a,c)||m._data(a,c,{empty:m.Callbacks("once memory").add(function(){m._removeData(a,b+"queue"),m._removeData(a,c)})})}}),m.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},W=/^(?:checkbox|radio)$/i;!function(){var a=y.createElement("input"),b=y.createElement("div"),c=y.createDocumentFragment();if(b.innerHTML="
      a",k.leadingWhitespace=3===b.firstChild.nodeType,k.tbody=!b.getElementsByTagName("tbody").length,k.htmlSerialize=!!b.getElementsByTagName("link").length,k.html5Clone="<:nav>"!==y.createElement("nav").cloneNode(!0).outerHTML,a.type="checkbox",a.checked=!0,c.appendChild(a),k.appendChecked=a.checked,b.innerHTML="",k.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,c.appendChild(b),b.innerHTML="",k.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,k.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){k.noCloneEvent=!1}),b.cloneNode(!0).click()),null==k.deleteExpando){k.deleteExpando=!0;try{delete b.test}catch(d){k.deleteExpando=!1}}}(),function(){var b,c,d=y.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(k[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),k[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var X=/^(?:input|select|textarea)$/i,Y=/^key/,Z=/^(?:mouse|pointer|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=/^([^.]*)(?:\.(.+)|)$/;function ab(){return!0}function bb(){return!1}function cb(){try{return y.activeElement}catch(a){}}m.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=m.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof m===K||a&&m.event.triggered===a.type?void 0:m.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(E)||[""],h=b.length;while(h--)f=_.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=m.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=m.event.special[o]||{},l=m.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&m.expr.match.needsContext.test(e),namespace:p.join(".")},i),(n=g[o])||(n=g[o]=[],n.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?n.splice(n.delegateCount++,0,l):n.push(l),m.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m.hasData(a)&&m._data(a);if(r&&(k=r.events)){b=(b||"").match(E)||[""],j=b.length;while(j--)if(h=_.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=m.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,n=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=n.length;while(f--)g=n[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(n.splice(f,1),g.selector&&n.delegateCount--,l.remove&&l.remove.call(a,g));i&&!n.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||m.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)m.event.remove(a,o+b[j],c,d,!0);m.isEmptyObject(k)&&(delete r.handle,m._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,n,o=[d||y],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||y,3!==d.nodeType&&8!==d.nodeType&&!$.test(p+m.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[m.expando]?b:new m.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:m.makeArray(c,[b]),k=m.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!m.isWindow(d)){for(i=k.delegateType||p,$.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocument||y)&&o.push(l.defaultView||l.parentWindow||a)}n=0;while((h=o[n++])&&!b.isPropagationStopped())b.type=n>1?i:k.bindType||p,f=(m._data(h,"events")||{})[b.type]&&m._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&m.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&m.acceptData(d)&&g&&d[p]&&!m.isWindow(d)){l=d[g],l&&(d[g]=null),m.event.triggered=p;try{d[p]()}catch(r){}m.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=m.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(m._data(this,"events")||{})[a.type]||[],k=m.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=m.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace))&&(a.handleObj=e,a.data=e.data,c=((m.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?m(c,this).index(i)>=0:m.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h]","i"),hb=/^\s+/,ib=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,jb=/<([\w:]+)/,kb=/\s*$/g,rb={option:[1,""],legend:[1,"

      ","
      "],area:[1,"",""],param:[1,"",""],thead:[1,"","
      "],tr:[2,"","
      "],col:[2,"","
      "],td:[3,"","
      "],_default:k.htmlSerialize?[0,"",""]:[1,"X
      ","
      "]},sb=db(y),tb=sb.appendChild(y.createElement("div"));rb.optgroup=rb.option,rb.tbody=rb.tfoot=rb.colgroup=rb.caption=rb.thead,rb.th=rb.td;function ub(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==K?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==K?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||m.nodeName(d,b)?f.push(d):m.merge(f,ub(d,b));return void 0===b||b&&m.nodeName(a,b)?m.merge([a],f):f}function vb(a){W.test(a.type)&&(a.defaultChecked=a.checked)}function wb(a,b){return m.nodeName(a,"table")&&m.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function xb(a){return a.type=(null!==m.find.attr(a,"type"))+"/"+a.type,a}function yb(a){var b=pb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function zb(a,b){for(var c,d=0;null!=(c=a[d]);d++)m._data(c,"globalEval",!b||m._data(b[d],"globalEval"))}function Ab(a,b){if(1===b.nodeType&&m.hasData(a)){var c,d,e,f=m._data(a),g=m._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)m.event.add(b,c,h[c][d])}g.data&&(g.data=m.extend({},g.data))}}function Bb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!k.noCloneEvent&&b[m.expando]){e=m._data(b);for(d in e.events)m.removeEvent(b,d,e.handle);b.removeAttribute(m.expando)}"script"===c&&b.text!==a.text?(xb(b).text=a.text,yb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),k.html5Clone&&a.innerHTML&&!m.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&W.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}m.extend({clone:function(a,b,c){var d,e,f,g,h,i=m.contains(a.ownerDocument,a);if(k.html5Clone||m.isXMLDoc(a)||!gb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(tb.innerHTML=a.outerHTML,tb.removeChild(f=tb.firstChild)),!(k.noCloneEvent&&k.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||m.isXMLDoc(a)))for(d=ub(f),h=ub(a),g=0;null!=(e=h[g]);++g)d[g]&&Bb(e,d[g]);if(b)if(c)for(h=h||ub(a),d=d||ub(f),g=0;null!=(e=h[g]);g++)Ab(e,d[g]);else Ab(a,f);return d=ub(f,"script"),d.length>0&&zb(d,!i&&ub(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,l,n=a.length,o=db(b),p=[],q=0;n>q;q++)if(f=a[q],f||0===f)if("object"===m.type(f))m.merge(p,f.nodeType?[f]:f);else if(lb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(jb.exec(f)||["",""])[1].toLowerCase(),l=rb[i]||rb._default,h.innerHTML=l[1]+f.replace(ib,"<$1>")+l[2],e=l[0];while(e--)h=h.lastChild;if(!k.leadingWhitespace&&hb.test(f)&&p.push(b.createTextNode(hb.exec(f)[0])),!k.tbody){f="table"!==i||kb.test(f)?""!==l[1]||kb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)m.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}m.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),k.appendChecked||m.grep(ub(p,"input"),vb),q=0;while(f=p[q++])if((!d||-1===m.inArray(f,d))&&(g=m.contains(f.ownerDocument,f),h=ub(o.appendChild(f),"script"),g&&zb(h),c)){e=0;while(f=h[e++])ob.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=m.expando,j=m.cache,l=k.deleteExpando,n=m.event.special;null!=(d=a[h]);h++)if((b||m.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)n[e]?m.event.remove(d,e):m.removeEvent(d,e,g.handle);j[f]&&(delete j[f],l?delete d[i]:typeof d.removeAttribute!==K?d.removeAttribute(i):d[i]=null,c.push(f))}}}),m.fn.extend({text:function(a){return V(this,function(a){return void 0===a?m.text(this):this.empty().append((this[0]&&this[0].ownerDocument||y).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=wb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?m.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||m.cleanData(ub(c)),c.parentNode&&(b&&m.contains(c.ownerDocument,c)&&zb(ub(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&m.cleanData(ub(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&m.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return m.clone(this,a,b)})},html:function(a){return V(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(fb,""):void 0;if(!("string"!=typeof a||mb.test(a)||!k.htmlSerialize&&gb.test(a)||!k.leadingWhitespace&&hb.test(a)||rb[(jb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(ib,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(m.cleanData(ub(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,m.cleanData(ub(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,l=this.length,n=this,o=l-1,p=a[0],q=m.isFunction(p);if(q||l>1&&"string"==typeof p&&!k.checkClone&&nb.test(p))return this.each(function(c){var d=n.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(l&&(i=m.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=m.map(ub(i,"script"),xb),f=g.length;l>j;j++)d=i,j!==o&&(d=m.clone(d,!0,!0),f&&m.merge(g,ub(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,m.map(g,yb),j=0;f>j;j++)d=g[j],ob.test(d.type||"")&&!m._data(d,"globalEval")&&m.contains(h,d)&&(d.src?m._evalUrl&&m._evalUrl(d.src):m.globalEval((d.text||d.textContent||d.innerHTML||"").replace(qb,"")));i=c=null}return this}}),m.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){m.fn[a]=function(a){for(var c,d=0,e=[],g=m(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),m(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Cb,Db={};function Eb(b,c){var d,e=m(c.createElement(b)).appendTo(c.body),f=a.getDefaultComputedStyle&&(d=a.getDefaultComputedStyle(e[0]))?d.display:m.css(e[0],"display");return e.detach(),f}function Fb(a){var b=y,c=Db[a];return c||(c=Eb(a,b),"none"!==c&&c||(Cb=(Cb||m("