From fc868aacee07389c39d2e77faa99749dc27b73f5 Mon Sep 17 00:00:00 2001 From: Joe Zhu Date: Thu, 15 Jun 2017 16:27:52 +0800 Subject: [PATCH 001/129] Add Azure eventhubs connector --- .../flink-connector-eventhubs/pom.xml | 141 +++++++++ .../connectors/eventhubs/EventFetcher.java | 210 ++++++++++++++ .../eventhubs/EventhubClientWrapper.java | 202 +++++++++++++ .../eventhubs/EventhubConsumerThread.java | 129 +++++++++ .../eventhubs/EventhubOffsetType.java | 11 + .../eventhubs/EventhubPartition.java | 68 +++++ .../eventhubs/EventhubPartitionState.java | 27 ++ .../eventhubs/FlinkEventHubConsumer.java | 268 ++++++++++++++++++ .../connectors/eventhubs/Handover.java | 108 +++++++ flink-connectors/pom.xml | 1 + 10 files changed, 1165 insertions(+) create mode 100644 flink-connectors/flink-connector-eventhubs/pom.xml create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventFetcher.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubClientWrapper.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubConsumerThread.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubOffsetType.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubPartition.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubPartitionState.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/Handover.java diff --git a/flink-connectors/flink-connector-eventhubs/pom.xml b/flink-connectors/flink-connector-eventhubs/pom.xml new file mode 100644 index 0000000000000..70e3c075a7ad8 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/pom.xml @@ -0,0 +1,141 @@ + + + + flink-connectors + org.apache.flink + 1.4-SNAPSHOT + + 4.0.0 + + flink-connector-azureeventhubs_${scala.binary.version} + flink-connector-azureeventhubs + + jar + + + + + + com.microsoft.azure + azure-eventhubs + 0.14.0 + provided + + + + org.apache.flink + flink-connector-kafka-base_${scala.binary.version} + ${project.version} + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + provided + + true + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + test + test-jar + + + + org.apache.flink + flink-tests_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.apache.flink + flink-metrics-jmx + ${project.version} + test + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-test-sources + + test-jar-no-fork + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + -Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit + + + + + + + diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventFetcher.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventFetcher.java new file mode 100644 index 0000000000000..894eb8dd60c90 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventFetcher.java @@ -0,0 +1,210 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +/** + * Created by jozh on 6/14/2017. + */ +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedValue; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + + + + +/** + * A fetcher that fetches data from Eventhub via the EventhubUtil. + * Eventhub offset is stored at flink checkpoint backend + * + * @param The type of elements produced by the fetcher. + */ +public class EventFetcher { + protected static final int NO_TIMESTAMPS_WATERMARKS = 0; + protected static final int PERIODIC_WATERMARKS = 1; + protected static final int PUNCTUATED_WATERMARKS = 2; + private static final Logger logger = LoggerFactory.getLogger(EventFetcher.class); + private volatile boolean running = true; + + private final KeyedDeserializationSchema deserializer; + private final Handover handover; + private final Properties eventhubProps; + private final EventhubConsumerThread consumerThread; + private final String taskNameWithSubtasks; + + + protected final SourceFunction.SourceContext sourceContext; + protected final Object checkpointLock; + private final Map subscribedPartitionStates; + protected final int timestampWatermarkMode; + protected final boolean useMetrics; + private volatile long maxWatermarkSoFar = Long.MIN_VALUE; + + public EventFetcher( + SourceFunction.SourceContext sourceContext, + Map assignedPartitionsWithInitialOffsets, + KeyedDeserializationSchema deserializer, + SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + ProcessingTimeService processTimerProvider, + long autoWatermarkInterval, + ClassLoader userCodeClassLoader, + String taskNameWithSubtasks, + Properties eventhubProps, + boolean useMetrics) throws Exception { + + this.sourceContext = Preconditions.checkNotNull(sourceContext); + this.deserializer = Preconditions.checkNotNull(deserializer); + this.eventhubProps = eventhubProps; + this.checkpointLock = sourceContext.getCheckpointLock(); + this.useMetrics = useMetrics; + this.taskNameWithSubtasks = taskNameWithSubtasks; + this.timestampWatermarkMode = getTimestampWatermarkMode(watermarksPeriodic, watermarksPunctuated); + this.subscribedPartitionStates = initializeSubscribedPartitionStates( + assignedPartitionsWithInitialOffsets, + timestampWatermarkMode, + watermarksPeriodic, watermarksPunctuated, + userCodeClassLoader); + + this.handover = new Handover(); + this.consumerThread = new EventhubConsumerThread(logger, + handover, + this.eventhubProps, + getFetcherName() + " for " + taskNameWithSubtasks, + this.subscribedPartitionStates.values().toArray(new EventhubPartitionState[this.subscribedPartitionStates.size()])); + + } + + public HashMap snapshotCurrentState() { + // this method assumes that the checkpoint lock is held + logger.debug("snapshot current offset state for subtask {}", taskNameWithSubtasks); + assert Thread.holdsLock(checkpointLock); + + HashMap state = new HashMap<>(subscribedPartitionStates.size()); + for (Map.Entry partition : subscribedPartitionStates.entrySet()){ + state.put(partition.getKey(), partition.getValue().getOffset()); + } + + return state; + } + + public void runFetchLoop() throws Exception{ + try { + final Handover handover = this.handover; + consumerThread.start(); + logger.info("Eventhub consumer thread started for substask {}", taskNameWithSubtasks); + + logger.info("Start fetcher loop to get data from eventhub and emit to flink for subtask {}", taskNameWithSubtasks); + while (running){ + final Tuple2> eventsTuple = handover.pollNext(); + for (EventData event : eventsTuple.f1){ + final T value = deserializer.deserialize(null, + event.getBytes(), + event.getSystemProperties().getPartitionKey(), + eventsTuple.f0.getParitionId(), + event.getSystemProperties().getSequenceNumber()); + + if (deserializer.isEndOfStream(value)){ + running = false; + break; + } + emitRecord(value, subscribedPartitionStates.get(eventsTuple.f0), event.getSystemProperties().getOffset()); + } + } + } + finally { + logger.warn("Stopping eventhub consumer thread of subtask {}, because something wrong when deserializing received event " + , taskNameWithSubtasks); + consumerThread.shutdown(); + } + + try { + consumerThread.join(); + logger.warn("Waiting eventhub consumer thread of subtask {} stopped", taskNameWithSubtasks); + } + catch (InterruptedException ex){ + Thread.currentThread().interrupt(); + } + + logger.info("EventFetcher of subtask {} stopped", taskNameWithSubtasks); + } + + public void cancel(){ + logger.info("EventFetcher of subtask {} canceled on demand", taskNameWithSubtasks); + running = false; + handover.close(); + consumerThread.shutdown(); + } + + protected void emitRecord(T record, EventhubPartitionState partitionState, String offset) throws Exception{ + if (record == null){ + synchronized (this.checkpointLock){ + partitionState.setOffset(offset); + } + return; + } + + if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS){ + synchronized (this.checkpointLock){ + sourceContext.collect(record); + partitionState.setOffset(offset); + } + } + else if (timestampWatermarkMode == PERIODIC_WATERMARKS){ + throw new Exception("Not implemented yet for AssignerWithPeriodicWatermarks"); + } + else { + throw new Exception("Not implemented yet for AssignerWithPunctuatedWatermarks"); + } + } + + protected String getFetcherName() { + return "Eventhubs Fetcher"; + } + + private int getTimestampWatermarkMode(SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated) + throws IllegalArgumentException { + if (watermarksPeriodic == null){ + if (watermarksPunctuated == null){ + return NO_TIMESTAMPS_WATERMARKS; + } + else { + return PUNCTUATED_WATERMARKS; + } + } + else { + if (watermarksPunctuated == null){ + return PERIODIC_WATERMARKS; + } + else { + throw new IllegalArgumentException("Cannot have both periodic and punctuated watermarks"); + } + } + } + + private Map initializeSubscribedPartitionStates( + Map assignedPartitionsWithInitialOffsets, + int timestampWatermarkMode, SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + ClassLoader userCodeClassLoader) { + + Map partitionsState = new HashMap<>(assignedPartitionsWithInitialOffsets.size()); + for (Map.Entry partition : assignedPartitionsWithInitialOffsets.entrySet()){ + partitionsState.put(partition.getKey(), new EventhubPartitionState(partition.getKey(), partition.getValue())); + logger.info("Assigned partition {}, offset is {}", partition.getKey(), partition.getValue()); + } + + return partitionsState; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubClientWrapper.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubClientWrapper.java new file mode 100644 index 0000000000000..272ba1e3d78ea --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubClientWrapper.java @@ -0,0 +1,202 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +/** + * Created by jozh on 6/14/2017. + */ +import org.apache.flink.util.Preconditions; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.ServiceBusException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.ExecutionException; + +/** + * + */ +public class EventhubClientWrapper implements Serializable { + private static final long serialVersionUID = -5319150387753930840L; + private static final Logger logger = LoggerFactory.getLogger(EventhubClientWrapper.class); + private EventHubClient eventHubClient; + private PartitionReceiver eventhubReceiver; + private ConnectionStringBuilder connectionString; + private String consumerGroup; + private Long receiverEpoch; + + private Duration receiverTimeout; + private EventhubOffsetType offsetType; + private String currentOffset; + private String partitionId; + + private final int minPrefetchCount = 10; + private int maxPrefetchCount = 999; + private int maxEventRate = 0; + private final Long defaultReceiverEpoch = -1L; + private final String defaultReceiverTimeout = "60000"; + + public void createReveiver(Properties eventhubParams, String partitionId) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10")); + this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM); + } + + public void createReveiver(Properties eventhubParams, String partitionId, String offset) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10")); + this.createReveiver(eventhubParams, partitionId, maxEventRate, offset); + } + + public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM); + } + + /*Will not implement a standalone offset store here, will leverage flink state to save the offset of eventhub*/ + public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate, String offset) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + if (eventhubParams.containsKey("eventhubs.uri") && eventhubParams.containsKey("eventhubs.namespace")) { + throw new IllegalArgumentException("Eventhubs URI and namespace cannot both be specified at the same time."); + } + + if (eventhubParams.containsKey("eventhubs.namespace")){ + this.connectionString = new ConnectionStringBuilder( + eventhubParams.getProperty("eventhubs.namespace"), + eventhubParams.getProperty("eventhubs.name"), + eventhubParams.getProperty("eventhubs.policyname"), + eventhubParams.getProperty("eventhubs.policykey")); + } + else if (eventhubParams.containsKey("eventhubs.uri")){ + this.connectionString = new ConnectionStringBuilder(new URI( + eventhubParams.getProperty("eventhubs.uri")), + eventhubParams.getProperty("eventhubs.name"), + eventhubParams.getProperty("eventhubs.policyname"), + eventhubParams.getProperty("eventhubs.policykey")); + } + else { + throw new IllegalArgumentException("Either Eventhubs URI or namespace nust be specified."); + } + + this.partitionId = Preconditions.checkNotNull(partitionId, "partitionId is no valid, cannot be null or empty"); + this.consumerGroup = eventhubParams.getProperty("eventhubs.consumergroup", EventHubClient.DEFAULT_CONSUMER_GROUP_NAME); + this.receiverEpoch = Long.parseLong(eventhubParams.getProperty("eventhubs.epoch", defaultReceiverEpoch.toString())); + this.receiverTimeout = Duration.ofMillis(Long.parseLong(eventhubParams.getProperty("eventhubs.receiver.timeout", defaultReceiverTimeout))); + this.offsetType = EventhubOffsetType.None; + this.currentOffset = PartitionReceiver.START_OF_STREAM; + + String previousOffset = offset; + + if (previousOffset != PartitionReceiver.START_OF_STREAM && previousOffset != null) { + + offsetType = EventhubOffsetType.PreviousCheckpoint; + currentOffset = previousOffset; + + } else if (eventhubParams.containsKey("eventhubs.filter.offset")) { + + offsetType = EventhubOffsetType.InputByteOffset; + currentOffset = eventhubParams.getProperty("eventhubs.filter.offset"); + + } else if (eventhubParams.containsKey("eventhubs.filter.enqueuetime")) { + + offsetType = EventhubOffsetType.InputTimeOffset; + currentOffset = eventhubParams.getProperty("eventhubs.filter.enqueuetime"); + } + + this.maxEventRate = maxEventRate; + + if (maxEventRate > 0 && maxEventRate < minPrefetchCount) { + maxPrefetchCount = minPrefetchCount; + } + else if (maxEventRate >= minPrefetchCount && maxEventRate < maxPrefetchCount) { + maxPrefetchCount = maxEventRate + 1; + } + else { + maxEventRate = maxPrefetchCount - 1; + } + + this.createReceiverInternal(); + } + + public Iterable receive () throws ExecutionException, InterruptedException { + return this.eventhubReceiver.receive(maxEventRate).get(); + } + + public void close(){ + logger.info("Close eventhub client on demand of partition {}", this.partitionId); + if (this.eventhubReceiver != null){ + try { + this.eventhubReceiver.closeSync(); + } + catch (ServiceBusException ex){ + logger.error("Close eventhub client of partition {} failed, reason: {}", this.partitionId, ex.getMessage()); + } + } + } + + private void createReceiverInternal() throws IOException, ServiceBusException{ + this.eventHubClient = EventHubClient.createFromConnectionStringSync(this.connectionString.toString()); + + switch (this.offsetType){ + case None: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case PreviousCheckpoint: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case InputByteOffset: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case InputTimeOffset: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset)), receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset))); + } + break; + } + } + + this.eventhubReceiver.setPrefetchCount(maxPrefetchCount); + this.eventhubReceiver.setReceiveTimeout(this.receiverTimeout); + logger.info("Successfully create eventhub receiver for partition {}, max_event_rate {}, max_prefetch_rate {}, receive_timeout {}, offset {}, ", + this.partitionId, + this.maxEventRate, + this.maxPrefetchCount, + this.receiverTimeout, + this.currentOffset); + } + + public Duration getReceiverTimeout() { + return receiverTimeout; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubConsumerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubConsumerThread.java new file mode 100644 index 0000000000000..81f0b59da1c60 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubConsumerThread.java @@ -0,0 +1,129 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +import org.apache.flink.api.java.tuple.Tuple2; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * Created by jozh on 5/24/2017. + * Cause eventhub client can only access one partition at one time, so here we should have multiple eventhub clients + * In this worker thread, it will receive event from each partition in round robin mode, any partition failed to retrive + * events will lead thread exception, and leverage flink HA framework to start from begining again + */ +public class EventhubConsumerThread extends Thread { + private final Logger logger; + private final Handover handover; + private final Properties eventhubProps; + private final EventhubPartitionState[] subscribedPartitionStates; + private final Map clients; + private volatile boolean running; + + public EventhubConsumerThread( + Logger logger, + Handover handover, + Properties eventhubProps, + String threadName, + EventhubPartitionState[] subscribedPartitionStates) throws Exception{ + + super(threadName); + setDaemon(true); + + this.logger = logger; + this.handover = handover; + this.eventhubProps = eventhubProps; + this.subscribedPartitionStates = subscribedPartitionStates; + this.running = true; + + this.clients = new HashMap<>(this.subscribedPartitionStates.length); + for (int i = 0; i < this.subscribedPartitionStates.length; i++){ + EventhubClientWrapper client = new EventhubClientWrapper(); + this.clients.put(this.subscribedPartitionStates[i], client); + } + } + + public void shutdown(){ + logger.info("Shutdown eventhub consumer thread {} on demand", this.getName()); + running = false; + handover.wakeupProducer(); + } + + @Override + public void run() { + if (!running){ + logger.info("Eventhub consumer thread is set to STOP, thread {} exit", this.getName()); + return; + } + + final Handover handover = this.handover; + + try { + logger.info("Starting create {} eventhub clients on {}", this.subscribedPartitionStates.length, this.getName()); + for (Map.Entry client : clients.entrySet()){ + EventhubPartitionState state = client.getKey(); + client.getValue().createReveiver(this.eventhubProps, Integer.toString(state.getPartition().getParitionId()), state.getOffset()); + } + } + catch (Throwable t){ + logger.error("Create eventhub client of {}, error: {}", this.getName(), t); + handover.reportError(t); + clearReceiveClients(); + return; + } + + try { + int currentClientIndex = 0; + while (running){ + EventhubPartitionState partitionState = subscribedPartitionStates[currentClientIndex]; + EventhubClientWrapper client = clients.get(partitionState); + Iterable events = client.receive(); + if (events != null){ + handover.produce(Tuple2.of(partitionState.getPartition(), events)); + logger.debug("Received event from {} on {}", partitionState.getPartition().toString(), this.getName()); + } + else { + logger.warn("Receive events from {} timeout, timeout set to {}, thread {}", + partitionState.getPartition().toString(), + client.getReceiverTimeout(), + this.getName()); + } + + currentClientIndex++; + currentClientIndex = currentClientIndex % subscribedPartitionStates.length; + } + } + catch (Throwable t){ + logger.error("Receving events error, {}", t); + handover.reportError(t); + } + finally { + logger.info("Exit from eventhub consumer thread, {}", this.getName()); + handover.close(); + clearReceiveClients(); + } + + logger.info("EventhubConsumerThread {} quit", this.getName()); + } + + private void clearReceiveClients(){ + if (clients == null){ + return; + } + + for (Map.Entry client : clients.entrySet()){ + try { + client.getValue().close(); + logger.info("Eventhub client for partition {} closed", client.getKey().getPartition().getParitionId()); + } + catch (Throwable t){ + logger.warn("Error while close eventhub client for partition {}, error is {}", + client.getKey().getPartition().getParitionId(), + t.getMessage()); + } + } + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubOffsetType.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubOffsetType.java new file mode 100644 index 0000000000000..5fecee47e2e03 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubOffsetType.java @@ -0,0 +1,11 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +/** + * Created by jozh on 5/22/2017. + */ +public enum EventhubOffsetType { + None, + PreviousCheckpoint, + InputByteOffset, + InputTimeOffset +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubPartition.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubPartition.java new file mode 100644 index 0000000000000..223cdaa9e614d --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubPartition.java @@ -0,0 +1,68 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; + +import java.io.Serializable; +import java.util.Properties; + +/** + * Created by jozh on 5/23/2017. + */ +public class EventhubPartition implements Serializable { + private static final long serialVersionUID = 134878919919793479L; + private final int cachedHash; + private final String policyName; + private final String policyKey; + private final String namespace; + private final String name; + + public int getParitionId() { + return paritionId; + } + + public String getPartitionName(){ + return namespace + "-" + name; + } + + private final int paritionId; + + public EventhubPartition(Properties props, int parition){ + this(props.getProperty("eventhubs.policyname"), + props.getProperty("eventhubs.policykey"), + props.getProperty("eventhubs.namespace"), + props.getProperty("eventhubs.name"), + parition); + } + + public EventhubPartition(String policyName, String policyKey, String namespace, String name, int paritionId){ + Preconditions.checkArgument(paritionId >= 0); + + this.policyName = Preconditions.checkNotNull(policyName); + this.policyKey = Preconditions.checkNotNull(policyKey); + this.name = Preconditions.checkNotNull(name); + this.namespace = Preconditions.checkNotNull(namespace); + this.paritionId = paritionId; + this.cachedHash = 31 * (this.namespace + this.name).hashCode() + paritionId; + } + + @Override + public String toString() { + return "EventhubPartition, namespace: " + this.namespace + + " name: " + this.name + + " partition: " + this.paritionId; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof EventhubPartition){ + return this.hashCode() == ((EventhubPartition) obj).hashCode(); + } + + return false; + } + + @Override + public int hashCode() { + return this.cachedHash; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubPartitionState.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubPartitionState.java new file mode 100644 index 0000000000000..c497c8f59062f --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/EventhubPartitionState.java @@ -0,0 +1,27 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +/** + * Created by jozh on 5/23/2017. + */ +public class EventhubPartitionState { + private final EventhubPartition partition; + private volatile String offset; + + public EventhubPartitionState(EventhubPartition partition, String offset){ + this.partition = partition; + this.offset = offset; + } + + public final String getOffset() { + return this.offset; + } + + public final void setOffset(String offset) { + this.offset = offset; + } + + public EventhubPartition getPartition() { + return this.partition; + } +} + diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java new file mode 100644 index 0000000000000..3af7bdbef7123 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -0,0 +1,268 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.DefaultOperatorStateBackend; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedValue; + +import com.microsoft.azure.eventhubs.PartitionReceiver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +/** + * Created by jozh on 5/22/2017. + * This class is used to create datastream from event hub + */ + +public class FlinkEventHubConsumer extends RichParallelSourceFunction implements + CheckpointedFunction, + ResultTypeQueryable, + CheckpointedRestoring> { + private static final long serialVersionUID = -3247976470793561346L; + protected static final Logger LOGGER = LoggerFactory.getLogger(FlinkEventHubConsumer.class); + protected static final String DEFAULTOFFSETSTATENAME = "flink.eventhub.offset"; + + protected final KeyedDeserializationSchema deserializer; + protected final Properties eventhubsProps; + protected final int partitionCount; + protected List> subscribedPartitions; + protected final String defaultEventhubInitOffset; + + private Map subscribedPartitionsToStartOffsets; + private SerializedValue> periodicWatermarkAssigner; + private SerializedValue> punctuatedWatermarkAssigner; + private transient ListState> offsetsStateForCheckpoint; + private transient volatile EventFetcher eventhubFetcher; + private transient volatile HashMap restoreToOffset; + private volatile boolean running = true; + + public FlinkEventHubConsumer(Properties eventhubsProps, DeserializationSchema deserializer){ + this(eventhubsProps, new KeyedDeserializationSchemaWrapper(deserializer)); + } + + public FlinkEventHubConsumer(Properties eventhubsProps, KeyedDeserializationSchema deserializer){ + Preconditions.checkNotNull(eventhubsProps); + Preconditions.checkNotNull(deserializer); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policyname")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policykey")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.namespace")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.name")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.partition.count")); + + this.eventhubsProps = eventhubsProps; + this.partitionCount = Integer.parseInt(eventhubsProps.getProperty("eventhubs.partition.count")); + this.deserializer = deserializer; + + String userDefinedOffset = eventhubsProps.getProperty("eventhubs.auto.offset"); + if (userDefinedOffset != null && userDefinedOffset.toLowerCase() == "lastest"){ + this.defaultEventhubInitOffset = PartitionReceiver.END_OF_STREAM; + } + else { + this.defaultEventhubInitOffset = PartitionReceiver.START_OF_STREAM; + } + + if (this.partitionCount <= 0){ + throw new IllegalArgumentException("eventhubs.partition.count must greater than 0"); + } + } + + @Override + public void open(Configuration parameters) throws Exception { + List eventhubPartitions = this.getAllEventhubPartitions(); + this.subscribedPartitionsToStartOffsets = new HashMap<>(eventhubPartitions.size()); + + if (this.restoreToOffset != null){ + for (EventhubPartition partition : eventhubPartitions){ + if (this.restoreToOffset.containsKey(partition)){ + this.subscribedPartitionsToStartOffsets.put(partition, restoreToOffset.get(partition)); + } + } + + LOGGER.info("Consumer subtask {} will start reading {} partitions with offsets in restored state: {}", + getRuntimeContext().getIndexOfThisSubtask(), + this.subscribedPartitionsToStartOffsets.size(), + this.subscribedPartitionsToStartOffsets); + } + else { + //If there is no restored state. Then all partitions to read from start, the offset is "-1". In the + //future eventhub supports specify offset, we modify here + //We assign partition to each subTask in round robin mode + int numParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); + int indexofThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); + for (int i = 0; i < eventhubPartitions.size(); i++) { + if (i % numParallelSubtasks == indexofThisSubtask) { + this.subscribedPartitionsToStartOffsets.put(eventhubPartitions.get(i), defaultEventhubInitOffset); + } + } + + LOGGER.info("Consumer subtask {} will start reading {} partitions with offsets: {}", + getRuntimeContext().getIndexOfThisSubtask(), + this.subscribedPartitionsToStartOffsets.size(), + this.subscribedPartitionsToStartOffsets); + } + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + if (this.subscribedPartitionsToStartOffsets == null || this.subscribedPartitionsToStartOffsets.size() == 0){ + throw new Exception("The partitions were not set for the consumer"); + } + + StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext(); + + if (!this.subscribedPartitionsToStartOffsets.isEmpty()){ + final EventFetcher fetcher = new EventFetcher(sourceContext, + subscribedPartitionsToStartOffsets, + deserializer, + periodicWatermarkAssigner, + punctuatedWatermarkAssigner, + runtimeContext.getProcessingTimeService(), + runtimeContext.getExecutionConfig().getAutoWatermarkInterval(), + runtimeContext.getUserCodeClassLoader(), + runtimeContext.getTaskNameWithSubtasks(), + eventhubsProps, + false); + + this.eventhubFetcher = fetcher; + if (!this.running){ + return; + } + + this.eventhubFetcher.runFetchLoop(); + } + else { + sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE)); + + final Object waitObj = new Object(); + while (this.running){ + try { + synchronized (waitObj){ + waitObj.wait(); + } + } + catch (InterruptedException ex){ + if (this.running){ + Thread.currentThread().interrupt(); + } + } + } + } + } + + @Override + public void close() throws Exception { + try { + this.cancel(); + } + finally { + super.close(); + } + } + + @Override + public void cancel() { + this.running = false; + + if (this.eventhubFetcher != null){ + this.eventhubFetcher.cancel(); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (!this.running){ + LOGGER.info("Consumer subtask {}: snapshotState() is called on the closed source", getRuntimeContext().getIndexOfThisSubtask()); + return; + } + + this.offsetsStateForCheckpoint.clear(); + final EventFetcher fetcher = this.eventhubFetcher; + if (fetcher == null){ + for (Map.Entry subscribedPartition : this.subscribedPartitionsToStartOffsets.entrySet()){ + this.offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue())); + } + } + else { + HashMap currentOffsets = fetcher.snapshotCurrentState(); + for (Map.Entry subscribedPartition : currentOffsets.entrySet()){ + this.offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue())); + } + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOGGER.info("Consumer subtask {}:Start init eventhub offset state", getRuntimeContext().getIndexOfThisSubtask()); + OperatorStateStore stateStore = context.getOperatorStateStore(); + /* this.offsetsStateForCheckpoint = stateStore + .getListState(new ListStateDescriptor>(DEFAULT_OFFSET_STATE_NAME, TypeInformation.of(new TypeHint>(){}))); +*/ + this.offsetsStateForCheckpoint = stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME); + if (context.isRestored()){ + if (this.restoreToOffset == null){ + this.restoreToOffset = new HashMap<>(); + for (Tuple2 offsetState : this.offsetsStateForCheckpoint.get()){ + this.restoreToOffset.put(offsetState.f0, offsetState.f1); + } + + LOGGER.info("Consumer subtask {}:Eventhub offset state is restored from checkpoint", getRuntimeContext().getIndexOfThisSubtask()); + } + else if (this.restoreToOffset.isEmpty()){ + this.restoreToOffset = null; + } + } + else { + LOGGER.info("Consumer subtask {}:No restore state for flink-eventhub-consumer", getRuntimeContext().getIndexOfThisSubtask()); + } + } + + @Override + public void restoreState(HashMap eventhubPartitionOffsets) throws Exception { + LOGGER.info("{} (taskIdx={}) restoring offsets from an older version.", + getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask()); + + this.restoreToOffset = eventhubPartitionOffsets; + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("{} (taskIdx={}) restored offsets from an older Flink version: {}", + getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), eventhubPartitionOffsets); + } + } + + @Override + public TypeInformation getProducedType() { + return this.deserializer.getProducedType(); + } + + private List getAllEventhubPartitions() { + List partitions = new ArrayList<>(); + for (int i = 0; i < this.partitionCount; i++){ + partitions.add(new EventhubPartition(this.eventhubsProps, i)); + } + + LOGGER.info("Consumer subtask {}:Create {} eventhub partitions info", getRuntimeContext().getIndexOfThisSubtask(), this.partitionCount); + return partitions; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/Handover.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/Handover.java new file mode 100644 index 0000000000000..322931afc57ba --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/Handover.java @@ -0,0 +1,108 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.ExceptionUtils; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.ThreadSafe; + +import java.io.Closeable; +import java.util.concurrent.ConcurrentLinkedQueue; + + +/** + * Created by jozh on 5/23/2017. + * Coming from flink kafka connector + */ +@ThreadSafe +public final class Handover implements Closeable { + private static final int MAX_EVENTS_BLOCK_IN_QUEUE = 1000; + private static final Logger logger = LoggerFactory.getLogger(Handover.class); + private ConcurrentLinkedQueue>> eventQueue = new ConcurrentLinkedQueue(); + + private volatile boolean allProducerWakeup = true; + + private Throwable error; + + public Tuple2> pollNext() throws Exception{ + logger.debug("###Begin to poll data from event cache queue"); + synchronized (eventQueue){ + while (eventQueue.isEmpty() && error == null){ + logger.debug("### No data in the msg queue, waiting... "); + eventQueue.wait(); + } + + logger.debug("### Get notified from eventhub consummer thread"); + Tuple2> events = eventQueue.poll(); + if (events != null && events.f0 != null && events.f1 != null){ + logger.debug("### Get event data from {}", events.f0.toString()); + int queueSize = eventQueue.size(); + if (queueSize < MAX_EVENTS_BLOCK_IN_QUEUE / 2){ + eventQueue.notifyAll(); + } + return events; + } + else { + ExceptionUtils.rethrowException(error, error.getMessage()); + return null; + } + } + } + + public void produce(final Tuple2> events) throws InterruptedException{ + if (events == null || events.f0 == null || events.f1 == null){ + logger.error("Received empty events from eventhub consumer thread"); + return; + } + + synchronized (eventQueue){ + while (eventQueue.size() > MAX_EVENTS_BLOCK_IN_QUEUE){ + logger.warn("Event queue is full, current size is {}", eventQueue.size()); + eventQueue.wait(); + } + + eventQueue.add(events); + eventQueue.notifyAll(); + logger.debug("Add received events into queue"); + } + } + + @Override + public void close() { + synchronized (eventQueue){ + logger.info("Close handover on demand"); + eventQueue.clear(); + if (error == null){ + error = new Throwable("Handover closed on command"); + } + + eventQueue.notifyAll(); + } + } + + public void reportError(Throwable t) { + if (t == null){ + return; + } + + synchronized (eventQueue){ + if (error == null){ + error = t; + } + eventQueue.clear(); + eventQueue.notifyAll(); + logger.info("Consumer thread report a error: {}", error.getMessage()); + } + } + + public void wakeupProducer() { + synchronized (eventQueue){ + logger.info("Wakeup producer on demand"); + eventQueue.clear(); + eventQueue.notifyAll(); + } + } +} diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index cbb48e0e9cac2..d9e356f31c9a3 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -53,6 +53,7 @@ under the License. flink-connector-nifi flink-connector-cassandra flink-connector-filesystem + flink-connector-eventhubs flink-test-utils-parent flink-libraries flink-scala-shell @@ -85,7 +85,7 @@ under the License. UTF-8 never-match-me - 2.4.1 + 2.8.0 From 896669805e62c71b27280a85a571eed66cca538d Mon Sep 17 00:00:00 2001 From: Joe Zhu Date: Mon, 26 Jun 2017 11:57:29 +0800 Subject: [PATCH 004/129] Fix retrive NM cores as -1 when NM core value is not set in yarn-site.xml --- .../org/apache/flink/yarn/AbstractYarnClusterDescriptor.java | 1 + 1 file changed, 1 insertion(+) 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 390b49ef63a1c..b1448ec7ce0de 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 @@ -332,6 +332,7 @@ private void isReadyForDeployment() throws YarnDeploymentException { // The number of cores can be configured in the config. // If not configured, it is set to the number of task slots int numYarnVcores = conf.getInt(YarnConfiguration.NM_VCORES, YarnConfiguration.DEFAULT_NM_VCORES); + numYarnVcores = numYarnVcores <= 0 ? YarnConfiguration.DEFAULT_NM_VCORES : numYarnVcores; int configuredVcores = flinkConfiguration.getInteger(ConfigConstants.YARN_VCORES, slots); // don't configure more than the maximum configured number of vcores if (configuredVcores > numYarnVcores) { From b7c1dfaa3175036287694520fc9bb1649707ef7d Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Wed, 21 Jun 2017 16:14:15 +0200 Subject: [PATCH 005/129] [FLINK-7055][blob] refactor getURL() to the more generic getFile() The fact that we always returned URL objects is a relic of the BlobServer's only use for URLClassLoader. Since we'd like to extend its use, returning File objects instead is more generic. This closes #4236. --- .../handlers/TaskManagerLogHandler.java | 2 +- .../apache/flink/runtime/blob/BlobCache.java | 17 ++++++------ .../apache/flink/runtime/blob/BlobServer.java | 18 ++++++------- .../flink/runtime/blob/BlobService.java | 8 +++--- .../flink/runtime/client/JobClient.java | 2 +- .../librarycache/BlobLibraryCacheManager.java | 2 +- .../runtime/blob/BlobCacheRetriesTest.java | 4 +-- .../runtime/blob/BlobCacheSuccessTest.java | 27 ++++++------------- .../runtime/blob/BlobServerDeleteTest.java | 4 +-- .../BlobLibraryCacheManagerTest.java | 8 +++--- .../BlobLibraryCacheRecoveryITCase.java | 6 ++--- 11 files changed, 44 insertions(+), 54 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java index b7fee2de9b64f..f175573aff844 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagerLogHandler.java @@ -208,7 +208,7 @@ protected void respondAsLeader(final ChannelHandlerContext ctx, final Routed rou lastSubmittedFile.put(taskManagerID, blobKey); } try { - return blobCache.getURL(blobKey).getFile(); + return blobCache.getFile(blobKey).getAbsolutePath(); } catch (IOException e) { throw new FlinkFutureException("Could not retrieve blob for " + blobKey + '.', e); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java index 32bd8fd83ed32..3e195374a71d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java @@ -30,7 +30,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.net.InetSocketAddress; -import java.net.URL; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.util.Preconditions.checkArgument; @@ -39,7 +38,7 @@ /** * The BLOB cache implements a local cache for content-addressable BLOBs. * - *

When requesting BLOBs through the {@link BlobCache#getURL} methods, the + *

When requesting BLOBs through the {@link BlobCache#getFile(BlobKey)} method, the * BLOB cache will first attempt to serve the file from its local cache. Only if * the local cache does not contain the desired BLOB, the BLOB cache will try to * download it from a distributed file system (if available) or the BLOB @@ -111,21 +110,22 @@ public BlobCache( } /** - * Returns the URL for the BLOB with the given key. The method will first attempt to serve + * Returns local copy of the file for the BLOB with the given key. The method will first attempt to serve * the BLOB from its local cache. If the BLOB is not in the cache, the method will try to download it * from this cache's BLOB server. * * @param requiredBlob The key of the desired BLOB. - * @return URL referring to the local storage location of the BLOB. + * @return file referring to the local storage location of the BLOB. * @throws IOException Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server. */ - public URL getURL(final BlobKey requiredBlob) throws IOException { + @Override + public File getFile(final BlobKey requiredBlob) throws IOException { checkArgument(requiredBlob != null, "BLOB key cannot be null."); final File localJarFile = BlobUtils.getStorageLocation(storageDir, requiredBlob); if (localJarFile.exists()) { - return localJarFile.toURI().toURL(); + return localJarFile; } // first try the distributed blob store (if available) @@ -136,7 +136,7 @@ public URL getURL(final BlobKey requiredBlob) throws IOException { } if (localJarFile.exists()) { - return localJarFile.toURI().toURL(); + return localJarFile; } // fallback: download from the BlobServer @@ -160,7 +160,7 @@ public URL getURL(final BlobKey requiredBlob) throws IOException { } // success, we finished - return localJarFile.toURI().toURL(); + return localJarFile; } catch (Throwable t) { String message = "Failed to fetch BLOB " + requiredBlob + " from " + serverAddress + @@ -188,6 +188,7 @@ public URL getURL(final BlobKey requiredBlob) throws IOException { * Deletes the file associated with the given key from the BLOB cache. * @param key referring to the file to be deleted */ + @Override public void delete(BlobKey key) throws IOException{ final File localFile = BlobUtils.getStorageLocation(storageDir, key); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java index ecb452701dcda..add9f7f7cdd4a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java @@ -34,7 +34,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.ServerSocket; -import java.net.URL; import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; @@ -334,22 +333,23 @@ public BlobClient createClient() throws IOException { } /** - * Method which retrieves the URL of a file associated with a blob key. The blob server looks - * the blob key up in its local storage. If the file exists, then the URL is returned. If the - * file does not exist, then a FileNotFoundException is thrown. + * Method which retrieves the local path of a file associated with a blob key. The blob server + * looks the blob key up in its local storage. If the file exists, it is returned. If the + * file does not exist, it is retrieved from the HA blob store (if available) or a + * FileNotFoundException is thrown. * * @param requiredBlob blob key associated with the requested file - * @return URL of the file - * @throws IOException + * @return file referring to the local storage location of the BLOB. + * @throws IOException Thrown if the file retrieval failed. */ @Override - public URL getURL(BlobKey requiredBlob) throws IOException { + public File getFile(BlobKey requiredBlob) throws IOException { checkArgument(requiredBlob != null, "BLOB key cannot be null."); final File localFile = BlobUtils.getStorageLocation(storageDir, requiredBlob); if (localFile.exists()) { - return localFile.toURI().toURL(); + return localFile; } else { try { @@ -361,7 +361,7 @@ public URL getURL(BlobKey requiredBlob) throws IOException { } if (localFile.exists()) { - return localFile.toURI().toURL(); + return localFile; } else { throw new FileNotFoundException("Local file " + localFile + " does not exist " + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java index c1447c849eeb3..1e56f26cabf3e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java @@ -19,8 +19,8 @@ package org.apache.flink.runtime.blob; import java.io.Closeable; +import java.io.File; import java.io.IOException; -import java.net.URL; /** * A simple store and retrieve binary large objects (BLOBs). @@ -28,14 +28,14 @@ public interface BlobService extends Closeable { /** - * Returns the URL of the file associated with the provided blob key. + * Returns the path to a local copy of the file associated with the provided blob key. * * @param key blob key associated with the requested file - * @return The URL to the file. + * @return The path to the file. * @throws java.io.FileNotFoundException when the path does not exist; * @throws IOException if any other error occurs when retrieving the file */ - URL getURL(BlobKey key) throws IOException; + File getFile(BlobKey key) throws IOException; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java index 19f0e2c635bf3..e3657ff54145e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java @@ -234,7 +234,7 @@ public static ClassLoader retrieveClassLoader( int pos = 0; for (BlobKey blobKey : props.requiredJarFiles()) { try { - allURLs[pos++] = blobClient.getURL(blobKey); + allURLs[pos++] = blobClient.getFile(blobKey).toURI().toURL(); } catch (Exception e) { try { blobClient.close(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index 0387725db0a0b..9aff6f94c7b67 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -254,7 +254,7 @@ private URL registerReferenceToBlobKeyAndGetURL(BlobKey key) throws IOException // it is important that we fetch the URL before increasing the counter. // in case the URL cannot be created (failed to fetch the BLOB), we have no stale counter try { - URL url = blobService.getURL(key); + URL url = blobService.getFile(key).toURI().toURL(); Integer references = blobKeyReferenceCounters.get(key); int newReferences = references == null ? 1 : references + 1; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java index 366b592a4b322..fe763fa20dad7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java @@ -115,7 +115,7 @@ private void testBlobFetchRetries(final Configuration config, final BlobStore bl cache = new BlobCache(serverAddress, config, new VoidBlobStore()); // trigger a download - it should fail the first two times, but retry, and succeed eventually - URL url = cache.getURL(key); + URL url = cache.getFile(key).toURI().toURL(); InputStream is = url.openStream(); try { byte[] received = new byte[data.length]; @@ -211,7 +211,7 @@ private void testBlobFetchWithTooManyFailures(final Configuration config, final // trigger a download - it should fail eventually try { - cache.getURL(key); + cache.getFile(key); fail("This should fail"); } catch (IOException e) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java index 51be1b044e21e..d06f76f167174 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java @@ -30,15 +30,12 @@ import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; -import java.net.URISyntaxException; -import java.net.URL; import java.util.ArrayList; import java.util.List; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * This class contains unit tests for the {@link BlobCache}. @@ -175,7 +172,7 @@ private void uploadFileGetTest(final Configuration config, boolean shutdownServe blobCache = new BlobCache(serverAddress, cacheConfig, blobStoreService); for (BlobKey blobKey : blobKeys) { - blobCache.getURL(blobKey); + blobCache.getFile(blobKey); } if (blobServer != null) { @@ -184,28 +181,20 @@ private void uploadFileGetTest(final Configuration config, boolean shutdownServe blobServer = null; } - final URL[] urls = new URL[blobKeys.size()]; + final File[] files = new File[blobKeys.size()]; for(int i = 0; i < blobKeys.size(); i++){ - urls[i] = blobCache.getURL(blobKeys.get(i)); + files[i] = blobCache.getFile(blobKeys.get(i)); } // Verify the result - assertEquals(blobKeys.size(), urls.length); + assertEquals(blobKeys.size(), files.length); - for (final URL url : urls) { + for (final File file : files) { + assertNotNull(file); - assertNotNull(url); - - try { - final File cachedFile = new File(url.toURI()); - - assertTrue(cachedFile.exists()); - assertEquals(buf.length, cachedFile.length()); - - } catch (URISyntaxException e) { - fail(e.getMessage()); - } + assertTrue(file.exists()); + assertEquals(buf.length, file.length()); } } finally { if (blobServer != null) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java index 5db956830e37a..ce4574b7b29ce 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java @@ -109,7 +109,7 @@ public void testDeleteSingleByBlobKey() { // delete a file directly on the server server.delete(key2); try { - server.getURL(key2); + server.getFile(key2); fail("BLOB should have been deleted"); } catch (IOException e) { @@ -209,7 +209,7 @@ public void testDeleteByBlobKeyFails() { server.delete(key); // the file should still be there - server.getURL(key); + server.getFile(key); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java index 606d8c9a0491e..476fdcbf3c458 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java @@ -108,13 +108,13 @@ public void testLibraryCacheManagerJobCleanup() throws IOException, InterruptedE assertEquals(0, checkFilesExist(keys, server, false)); try { - server.getURL(keys.get(0)); + server.getFile(keys.get(0)); fail("name-addressable BLOB should have been deleted"); } catch (IOException e) { // expected } try { - server.getURL(keys.get(1)); + server.getFile(keys.get(1)); fail("name-addressable BLOB should have been deleted"); } catch (IOException e) { // expected @@ -150,7 +150,7 @@ public void testLibraryCacheManagerJobCleanup() throws IOException, InterruptedE * @param doThrow * whether exceptions should be ignored (false), or throws (true) * - * @return number of files we were able to retrieve via {@link BlobService#getURL(BlobKey)} + * @return number of files we were able to retrieve via {@link BlobService#getFile(BlobKey)} */ private static int checkFilesExist( List keys, BlobService blobService, boolean doThrow) @@ -159,7 +159,7 @@ private static int checkFilesExist( for (BlobKey key : keys) { try { - blobService.getURL(key); + blobService.getFile(key); ++numFiles; } catch (IOException e) { if (doThrow) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java index e5efd19ba59f7..b19835bc8c4a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java @@ -107,7 +107,7 @@ public void testRecoveryRegisterAndDownload() throws Exception { libServer[0].registerTask(jobId, executionId, keys, Collections.emptyList()); // Verify key 1 - File f = new File(cache.getURL(keys.get(0)).toURI()); + File f = cache.getFile(keys.get(0)); assertEquals(expected.length, f.length()); try (FileInputStream fis = new FileInputStream(f)) { @@ -126,7 +126,7 @@ public void testRecoveryRegisterAndDownload() throws Exception { libCache = new BlobLibraryCacheManager(cache, 3600 * 1000); // Verify key 1 - f = new File(cache.getURL(keys.get(0)).toURI()); + f = cache.getFile(keys.get(0)); assertEquals(expected.length, f.length()); try (FileInputStream fis = new FileInputStream(f)) { @@ -138,7 +138,7 @@ public void testRecoveryRegisterAndDownload() throws Exception { } // Verify key 2 - f = new File(cache.getURL(keys.get(1)).toURI()); + f = cache.getFile(keys.get(1)); assertEquals(256, f.length()); try (FileInputStream fis = new FileInputStream(f)) { From 0a19c456ac7781d94eb0aaaf8f2ac73d0157bacb Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Wed, 21 Jun 2017 18:04:43 +0200 Subject: [PATCH 006/129] [FLINK-7056][blob] add API to allow job-related BLOBs to be stored [FLINK-7056][blob] refactor the new API for job-related BLOBs For a cleaner API, instead of having a nullable jobId parameter, use two methods: one for job-related BLOBs, another for job-unrelated ones. This closes #4237. --- .../apache/flink/runtime/blob/BlobCache.java | 167 ++++++-- .../apache/flink/runtime/blob/BlobClient.java | 361 ++++++++++++------ .../apache/flink/runtime/blob/BlobServer.java | 132 +++++-- .../runtime/blob/BlobServerConnection.java | 228 ++++++----- .../runtime/blob/BlobServerProtocol.java | 12 +- .../flink/runtime/blob/BlobService.java | 29 +- .../apache/flink/runtime/blob/BlobStore.java | 6 +- .../apache/flink/runtime/blob/BlobUtils.java | 151 +++++--- .../apache/flink/runtime/blob/BlobView.java | 5 +- .../runtime/blob/FileSystemBlobStore.java | 14 +- .../flink/runtime/blob/VoidBlobStore.java | 7 +- .../flink/runtime/client/JobClient.java | 1 + .../flink/runtime/jobgraph/JobGraph.java | 1 + .../runtime/blob/BlobCacheRetriesTest.java | 103 ++++- .../runtime/blob/BlobCacheSuccessTest.java | 100 ++++- .../flink/runtime/blob/BlobClientTest.java | 36 +- .../runtime/blob/BlobRecoveryITCase.java | 34 +- .../runtime/blob/BlobServerDeleteTest.java | 143 +++++-- .../flink/runtime/blob/BlobServerGetTest.java | 133 ++++++- .../flink/runtime/blob/BlobServerPutTest.java | 212 ++++++++-- .../flink/runtime/blob/BlobUtilsTest.java | 22 +- .../BlobLibraryCacheManagerTest.java | 67 ++-- .../BlobLibraryCacheRecoveryITCase.java | 9 +- .../runtime/jobmanager/JobSubmitTest.java | 7 +- 24 files changed, 1503 insertions(+), 477 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java index 3e195374a71d7..29f7706154de2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java @@ -18,12 +18,15 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.FileUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -38,7 +41,7 @@ /** * The BLOB cache implements a local cache for content-addressable BLOBs. * - *

When requesting BLOBs through the {@link BlobCache#getFile(BlobKey)} method, the + *

When requesting BLOBs through the {@link BlobCache#getFile} methods, the * BLOB cache will first attempt to serve the file from its local cache. Only if * the local cache does not contain the desired BLOB, the BLOB cache will try to * download it from a distributed file system (if available) or the BLOB @@ -91,7 +94,7 @@ public BlobCache( // configure and create the storage directory String storageDirectory = blobClientConfig.getString(BlobServerOptions.STORAGE_DIRECTORY); - this.storageDir = BlobUtils.initStorageDirectory(storageDirectory); + this.storageDir = BlobUtils.initLocalStorageDirectory(storageDirectory); LOG.info("Created BLOB cache storage directory " + storageDir); // configure the number of fetch retries @@ -110,19 +113,66 @@ public BlobCache( } /** - * Returns local copy of the file for the BLOB with the given key. The method will first attempt to serve - * the BLOB from its local cache. If the BLOB is not in the cache, the method will try to download it - * from this cache's BLOB server. + * Returns local copy of the (job-unrelated) file for the BLOB with the given key. + *

+ * The method will first attempt to serve the BLOB from its local cache. If the BLOB is not in + * the cache, the method will try to download it from this cache's BLOB server. + * + * @param key + * The key of the desired BLOB. * - * @param requiredBlob The key of the desired BLOB. * @return file referring to the local storage location of the BLOB. - * @throws IOException Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server. + * + * @throws IOException + * Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server. */ @Override - public File getFile(final BlobKey requiredBlob) throws IOException { + public File getFile(BlobKey key) throws IOException { + return getFileInternal(null, key); + } + + /** + * Returns local copy of the file for the BLOB with the given key. + *

+ * The method will first attempt to serve the BLOB from its local cache. If the BLOB is not in + * the cache, the method will try to download it from this cache's BLOB server. + * + * @param jobId + * ID of the job this blob belongs to + * @param key + * The key of the desired BLOB. + * + * @return file referring to the local storage location of the BLOB. + * + * @throws IOException + * Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server. + */ + @Override + public File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException { + checkNotNull(jobId); + return getFileInternal(jobId, key); + } + + /** + * Returns local copy of the file for the BLOB with the given key. + *

+ * The method will first attempt to serve the BLOB from its local cache. If the BLOB is not in + * the cache, the method will try to download it from this cache's BLOB server. + * + * @param jobId + * ID of the job this blob belongs to (or null if job-unrelated) + * @param requiredBlob + * The key of the desired BLOB. + * + * @return file referring to the local storage location of the BLOB. + * + * @throws IOException + * Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server. + */ + private File getFileInternal(@Nullable JobID jobId, BlobKey requiredBlob) throws IOException { checkArgument(requiredBlob != null, "BLOB key cannot be null."); - final File localJarFile = BlobUtils.getStorageLocation(storageDir, requiredBlob); + final File localJarFile = BlobUtils.getStorageLocation(storageDir, jobId, requiredBlob); if (localJarFile.exists()) { return localJarFile; @@ -130,7 +180,7 @@ public File getFile(final BlobKey requiredBlob) throws IOException { // first try the distributed blob store (if available) try { - blobView.get(requiredBlob, localJarFile); + blobView.get(jobId, requiredBlob, localJarFile); } catch (Exception e) { LOG.info("Failed to copy from blob store. Downloading from BLOB server instead.", e); } @@ -141,14 +191,14 @@ public File getFile(final BlobKey requiredBlob) throws IOException { // fallback: download from the BlobServer final byte[] buf = new byte[BlobServerProtocol.BUFFER_SIZE]; - LOG.info("Downloading {} from {}", requiredBlob, serverAddress); + LOG.info("Downloading {}/{} from {}", jobId, requiredBlob, serverAddress); // loop over retries int attempt = 0; while (true) { try ( final BlobClient bc = new BlobClient(serverAddress, blobClientConfig); - final InputStream is = bc.get(requiredBlob); + final InputStream is = bc.getInternal(jobId, requiredBlob); final OutputStream os = new FileOutputStream(localJarFile) ) { while (true) { @@ -163,7 +213,7 @@ public File getFile(final BlobKey requiredBlob) throws IOException { return localJarFile; } catch (Throwable t) { - String message = "Failed to fetch BLOB " + requiredBlob + " from " + serverAddress + + String message = "Failed to fetch BLOB " + jobId + "/" + requiredBlob + " from " + serverAddress + " and store it under " + localJarFile.getAbsolutePath(); if (attempt < numFetchRetries) { if (LOG.isDebugEnabled()) { @@ -179,41 +229,110 @@ public File getFile(final BlobKey requiredBlob) throws IOException { // retry ++attempt; - LOG.info("Downloading {} from {} (retry {})", requiredBlob, serverAddress, attempt); + LOG.info("Downloading {}/{} from {} (retry {})", jobId, requiredBlob, serverAddress, attempt); } } // end loop over retries } /** - * Deletes the file associated with the given key from the BLOB cache. - * @param key referring to the file to be deleted + * Deletes the (job-unrelated) file associated with the blob key in this BLOB cache. + * + * @param key + * blob key associated with the file to be deleted + * + * @throws IOException */ @Override - public void delete(BlobKey key) throws IOException{ - final File localFile = BlobUtils.getStorageLocation(storageDir, key); + public void delete(BlobKey key) throws IOException { + deleteInternal(null, key); + } + /** + * Deletes the file associated with the blob key in this BLOB cache. + * + * @param jobId + * ID of the job this blob belongs to + * @param key + * blob key associated with the file to be deleted + * + * @throws IOException + */ + @Override + public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException { + checkNotNull(jobId); + deleteInternal(jobId, key); + } + + /** + * Deletes the file associated with the blob key in this BLOB cache. + * + * @param jobId + * ID of the job this blob belongs to (or null if job-unrelated) + * @param key + * blob key associated with the file to be deleted + * + * @throws IOException + */ + private void deleteInternal(@Nullable JobID jobId, BlobKey key) throws IOException{ + final File localFile = BlobUtils.getStorageLocation(storageDir, jobId, key); if (!localFile.delete() && localFile.exists()) { LOG.warn("Failed to delete locally cached BLOB {} at {}", key, localFile.getAbsolutePath()); } } /** - * Deletes the file associated with the given key from the BLOB cache and + * Deletes the (job-unrelated) file associated with the given key from the BLOB cache and * BLOB server. * - * @param key referring to the file to be deleted + * @param key + * referring to the file to be deleted + * * @throws IOException - * thrown if an I/O error occurs while transferring the request to - * the BLOB server or if the BLOB server cannot delete the file + * thrown if an I/O error occurs while transferring the request to the BLOB server or if the + * BLOB server cannot delete the file */ public void deleteGlobal(BlobKey key) throws IOException { + deleteGlobalInternal(null, key); + } + + /** + * Deletes the file associated with the given key from the BLOB cache and BLOB server. + * + * @param jobId + * ID of the job this blob belongs to + * @param key + * referring to the file to be deleted + * + * @throws IOException + * thrown if an I/O error occurs while transferring the request to the BLOB server or if the + * BLOB server cannot delete the file + */ + public void deleteGlobal(@Nonnull JobID jobId, BlobKey key) throws IOException { + checkNotNull(jobId); + deleteGlobalInternal(jobId, key); + } + + /** + * Deletes the file associated with the given key from the BLOB cache and + * BLOB server. + * + * @param jobId + * ID of the job this blob belongs to (or null if job-unrelated) + * @param key + * referring to the file to be deleted + * + * @throws IOException + * thrown if an I/O error occurs while transferring the request to the BLOB server or if the + * BLOB server cannot delete the file + */ + private void deleteGlobalInternal(@Nullable JobID jobId, BlobKey key) throws IOException { // delete locally - delete(key); + deleteInternal(jobId, key); // then delete on the BLOB server // (don't use the distributed storage directly - this way the blob // server is aware of the delete operation, too) try (BlobClient bc = createClient()) { - bc.delete(key); + bc.deleteInternal(jobId, key); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java index 0882ec3905820..9a2f59e31cf14 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; @@ -29,6 +30,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLParameters; import javax.net.ssl.SSLSocket; @@ -46,7 +49,8 @@ import java.util.List; import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE; -import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE; +import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_FOR_JOB; +import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_NO_JOB; import static org.apache.flink.runtime.blob.BlobServerProtocol.DELETE_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION; @@ -55,7 +59,7 @@ import static org.apache.flink.runtime.blob.BlobUtils.readFully; import static org.apache.flink.runtime.blob.BlobUtils.readLength; import static org.apache.flink.runtime.blob.BlobUtils.writeLength; -import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * The BLOB client can communicate with the BLOB server and either upload (PUT), download (GET), @@ -75,6 +79,7 @@ public final class BlobClient implements Closeable { * the network address of the BLOB server * @param clientConfig * additional configuration like SSL parameters required to connect to the blob server + * * @throws IOException * thrown if the connection to the BLOB server could not be established */ @@ -130,22 +135,65 @@ public boolean isClosed() { // -------------------------------------------------------------------------------------------- /** - * Downloads the BLOB identified by the given BLOB key from the BLOB server. If no such BLOB exists on the server, a - * {@link FileNotFoundException} is thrown. - * + * Downloads the (job-unrelated) BLOB identified by the given BLOB key from the BLOB server. + * * @param blobKey - * the BLOB key identifying the BLOB to download + * blob key associated with the requested file + * * @return an input stream to read the retrieved data from + * + * @throws FileNotFoundException + * if there is no such file; * @throws IOException - * thrown if an I/O error occurs during the download + * if an I/O error occurs during the download */ public InputStream get(BlobKey blobKey) throws IOException { + return getInternal(null, blobKey); + } + + /** + * Downloads the BLOB identified by the given BLOB key from the BLOB server. + * + * @param jobId + * ID of the job this blob belongs to + * @param blobKey + * blob key associated with the requested file + * + * @return an input stream to read the retrieved data from + * + * @throws FileNotFoundException + * if there is no such file; + * @throws IOException + * if an I/O error occurs during the download + */ + public InputStream get(@Nonnull JobID jobId, BlobKey blobKey) throws IOException { + checkNotNull(jobId); + return getInternal(jobId, blobKey); + } + + /** + * Downloads the BLOB identified by the given BLOB key from the BLOB server. + * + * @param jobId + * ID of the job this blob belongs to (or null if job-unrelated) + * @param blobKey + * blob key associated with the requested file + * + * @return an input stream to read the retrieved data from + * + * @throws FileNotFoundException + * if there is no such file; + * @throws IOException + * if an I/O error occurs during the download + */ + InputStream getInternal(@Nullable JobID jobId, BlobKey blobKey) throws IOException { if (this.socket.isClosed()) { throw new IllegalStateException("BLOB Client is not connected. " + "Client has been shut down or encountered an error before."); } if (LOG.isDebugEnabled()) { - LOG.debug(String.format("GET content addressable BLOB %s from %s", blobKey, socket.getLocalSocketAddress())); + LOG.debug("GET BLOB {}/{} from {}.", jobId, blobKey, + socket.getLocalSocketAddress()); } try { @@ -153,8 +201,8 @@ public InputStream get(BlobKey blobKey) throws IOException { InputStream is = this.socket.getInputStream(); // Send GET header - sendGetHeader(os, null, blobKey); - receiveAndCheckResponse(is); + sendGetHeader(os, jobId, blobKey); + receiveAndCheckGetResponse(is); return new BlobInputStream(is, blobKey); } @@ -169,29 +217,40 @@ public InputStream get(BlobKey blobKey) throws IOException { * * @param outputStream * the output stream to write the header data to - * @param jobID - * the job ID identifying the BLOB to download or null to indicate the BLOB key should be used - * to identify the BLOB on the server instead + * @param jobId + * ID of the job this blob belongs to (or null if job-unrelated) * @param blobKey - * the BLOB key to identify the BLOB to download if either the job ID or the regular key are - * null + * blob key associated with the requested file + * * @throws IOException * thrown if an I/O error occurs while writing the header data to the output stream */ - private void sendGetHeader(OutputStream outputStream, JobID jobID, BlobKey blobKey) throws IOException { - checkArgument(jobID == null); + private static void sendGetHeader(OutputStream outputStream, @Nullable JobID jobId, BlobKey blobKey) throws IOException { + checkNotNull(blobKey); // Signal type of operation outputStream.write(GET_OPERATION); - // Check if GET should be done in content-addressable manner - if (jobID == null) { - outputStream.write(CONTENT_ADDRESSABLE); - blobKey.writeToOutputStream(outputStream); + // Send job ID and key + if (jobId == null) { + outputStream.write(CONTENT_NO_JOB); + } else { + outputStream.write(CONTENT_FOR_JOB); + outputStream.write(jobId.getBytes()); } + blobKey.writeToOutputStream(outputStream); } - private void receiveAndCheckResponse(InputStream is) throws IOException { + /** + * Reads the response from the input stream and throws in case of errors + * + * @param is + * stream to read from + * + * @throws IOException + * if the response is an error or reading the response failed + */ + private static void receiveAndCheckGetResponse(InputStream is) throws IOException { int response = is.read(); if (response < 0) { throw new EOFException("Premature end of response"); @@ -211,82 +270,111 @@ else if (response != RETURN_OKAY) { // -------------------------------------------------------------------------------------------- /** - * Uploads the data of the given byte array to the BLOB server in a content-addressable manner. + * Uploads the data of the given byte array for the given job to the BLOB server. * + * @param jobId + * the ID of the job the BLOB belongs to (or null if job-unrelated) * @param value - * the buffer to upload + * the buffer to upload + * * @return the computed BLOB key identifying the BLOB on the server + * * @throws IOException - * thrown if an I/O error occurs while uploading the data to the BLOB server + * thrown if an I/O error occurs while uploading the data to the BLOB server */ - public BlobKey put(byte[] value) throws IOException { - return put(value, 0, value.length); + @VisibleForTesting + public BlobKey put(@Nullable JobID jobId, byte[] value) throws IOException { + return put(jobId, value, 0, value.length); } /** - * Uploads data from the given byte array to the BLOB server in a content-addressable manner. + * Uploads data from the given byte array for the given job to the BLOB server. * + * @param jobId + * the ID of the job the BLOB belongs to (or null if job-unrelated) * @param value - * the buffer to upload data from + * the buffer to upload data from * @param offset - * the read offset within the buffer + * the read offset within the buffer * @param len - * the number of bytes to upload from the buffer + * the number of bytes to upload from the buffer + * * @return the computed BLOB key identifying the BLOB on the server + * * @throws IOException - * thrown if an I/O error occurs while uploading the data to the BLOB server + * thrown if an I/O error occurs while uploading the data to the BLOB server */ - public BlobKey put(byte[] value, int offset, int len) throws IOException { - return putBuffer(null, value, offset, len); + @VisibleForTesting + public BlobKey put(@Nullable JobID jobId, byte[] value, int offset, int len) throws IOException { + return putBuffer(jobId, value, offset, len); } /** - * Uploads the data from the given input stream to the BLOB server in a content-addressable manner. + * Uploads the (job-unrelated) data from the given input stream to the BLOB server. * * @param inputStream - * the input stream to read the data from + * the input stream to read the data from + * * @return the computed BLOB key identifying the BLOB on the server + * * @throws IOException - * thrown if an I/O error occurs while reading the data from the input stream or uploading the data to the - * BLOB server + * thrown if an I/O error occurs while reading the data from the input stream or uploading the + * data to the BLOB server */ public BlobKey put(InputStream inputStream) throws IOException { return putInputStream(null, inputStream); } + /** + * Uploads the data from the given input stream for the given job to the BLOB server. + * + * @param jobId + * ID of the job this blob belongs to + * @param inputStream + * the input stream to read the data from + * + * @return the computed BLOB key identifying the BLOB on the server + * + * @throws IOException + * thrown if an I/O error occurs while reading the data from the input stream or uploading the + * data to the BLOB server + */ + public BlobKey put(@Nonnull JobID jobId, InputStream inputStream) throws IOException { + checkNotNull(jobId); + return putInputStream(jobId, inputStream); + } + /** * Uploads data from the given byte buffer to the BLOB server. * * @param jobId - * the ID of the job the BLOB belongs to or null to store the BLOB in a content-addressable - * manner + * the ID of the job the BLOB belongs to (or null if job-unrelated) * @param value - * the buffer to read the data from + * the buffer to read the data from * @param offset - * the read offset within the buffer + * the read offset within the buffer * @param len - * the number of bytes to read from the buffer - * @return the computed BLOB key if the BLOB has been stored in a content-addressable manner, null - * otherwise + * the number of bytes to read from the buffer + * + * @return the computed BLOB key of the uploaded BLOB + * * @throws IOException - * thrown if an I/O error occurs while uploading the data to the BLOB server + * thrown if an I/O error occurs while uploading the data to the BLOB server */ - private BlobKey putBuffer(JobID jobId, byte[] value, int offset, int len) throws IOException { + private BlobKey putBuffer(@Nullable JobID jobId, byte[] value, int offset, int len) throws IOException { if (this.socket.isClosed()) { throw new IllegalStateException("BLOB Client is not connected. " + "Client has been shut down or encountered an error before."); } + checkNotNull(value); if (LOG.isDebugEnabled()) { - if (jobId == null) { - LOG.debug(String.format("PUT content addressable BLOB buffer (%d bytes) to %s", - len, socket.getLocalSocketAddress())); - } + LOG.debug("PUT BLOB buffer ({} bytes) to {}.", len, socket.getLocalSocketAddress()); } try { final OutputStream os = this.socket.getOutputStream(); - final MessageDigest md = jobId == null ? BlobUtils.createMessageDigest() : null; + final MessageDigest md = BlobUtils.createMessageDigest(); // Send the PUT header sendPutHeader(os, jobId); @@ -295,15 +383,15 @@ private BlobKey putBuffer(JobID jobId, byte[] value, int offset, int len) throws int remainingBytes = len; while (remainingBytes > 0) { + // want a common code path for byte[] and InputStream at the BlobServer + // -> since for InputStream we don't know a total size beforehand, send lengths iteratively final int bytesToSend = Math.min(BUFFER_SIZE, remainingBytes); writeLength(bytesToSend, os); os.write(value, offset, bytesToSend); - // Update the message digest if necessary - if (md != null) { - md.update(value, offset, bytesToSend); - } + // Update the message digest + md.update(value, offset, bytesToSend); remainingBytes -= bytesToSend; offset += bytesToSend; @@ -313,7 +401,7 @@ private BlobKey putBuffer(JobID jobId, byte[] value, int offset, int len) throws // Receive blob key and compare final InputStream is = this.socket.getInputStream(); - return receivePutResponseAndCompare(is, md); + return receiveAndCheckPutResponse(is, md); } catch (Throwable t) { BlobUtils.closeSilently(socket, LOG); @@ -325,37 +413,36 @@ private BlobKey putBuffer(JobID jobId, byte[] value, int offset, int len) throws * Uploads data from the given input stream to the BLOB server. * * @param jobId - * the ID of the job the BLOB belongs to or null to store the BLOB in a content-addressable - * manner + * the ID of the job the BLOB belongs to (or null if job-unrelated) * @param inputStream - * the input stream to read the data from - * @return he computed BLOB key if the BLOB has been stored in a content-addressable manner, null - * otherwise + * the input stream to read the data from + * + * @return the computed BLOB key of the uploaded BLOB + * * @throws IOException - * thrown if an I/O error occurs while uploading the data to the BLOB server + * thrown if an I/O error occurs while uploading the data to the BLOB server */ - private BlobKey putInputStream(JobID jobId, InputStream inputStream) throws IOException { + private BlobKey putInputStream(@Nullable JobID jobId, InputStream inputStream) throws IOException { if (this.socket.isClosed()) { throw new IllegalStateException("BLOB Client is not connected. " + "Client has been shut down or encountered an error before."); } + checkNotNull(inputStream); if (LOG.isDebugEnabled()) { - if (jobId == null) { - LOG.debug(String.format("PUT content addressable BLOB stream to %s", - socket.getLocalSocketAddress())); - } + LOG.debug("PUT BLOB stream to {}.", socket.getLocalSocketAddress()); } try { final OutputStream os = this.socket.getOutputStream(); - final MessageDigest md = jobId == null ? BlobUtils.createMessageDigest() : null; + final MessageDigest md = BlobUtils.createMessageDigest(); final byte[] xferBuf = new byte[BUFFER_SIZE]; // Send the PUT header sendPutHeader(os, jobId); while (true) { + // since we don't know a total size here, send lengths iteratively final int read = inputStream.read(xferBuf); if (read < 0) { // we are done. send a -1 and be done @@ -365,15 +452,13 @@ private BlobKey putInputStream(JobID jobId, InputStream inputStream) throws IOEx if (read > 0) { writeLength(read, os); os.write(xferBuf, 0, read); - if (md != null) { - md.update(xferBuf, 0, read); - } + md.update(xferBuf, 0, read); } } // Receive blob key and compare final InputStream is = this.socket.getInputStream(); - return receivePutResponseAndCompare(is, md); + return receiveAndCheckPutResponse(is, md); } catch (Throwable t) { BlobUtils.closeSilently(socket, LOG); @@ -381,16 +466,25 @@ private BlobKey putInputStream(JobID jobId, InputStream inputStream) throws IOEx } } - private BlobKey receivePutResponseAndCompare(InputStream is, MessageDigest md) throws IOException { + /** + * Reads the response from the input stream and throws in case of errors + * + * @param is + * stream to read from + * @param md + * message digest to check the response against + * + * @throws IOException + * if the response is an error, the message digest does not match or reading the response + * failed + */ + private static BlobKey receiveAndCheckPutResponse(InputStream is, MessageDigest md) + throws IOException { int response = is.read(); if (response < 0) { throw new EOFException("Premature end of response"); } else if (response == RETURN_OKAY) { - if (md == null) { - // not content addressable - return null; - } BlobKey remoteKey = BlobKey.readFromInputStream(is); BlobKey localKey = new BlobKey(md.digest()); @@ -412,24 +506,24 @@ else if (response == RETURN_ERROR) { /** * Constructs and writes the header data for a PUT request to the given output stream. - * NOTE: If the jobId and key are null, we send the data to the content addressable section. * * @param outputStream - * the output stream to write the PUT header data to - * @param jobID - * the ID of job the BLOB belongs to or null to indicate the upload of a - * content-addressable BLOB + * the output stream to write the PUT header data to + * @param jobId + * the ID of job the BLOB belongs to (or null if job-unrelated) + * * @throws IOException - * thrown if an I/O error occurs while writing the header data to the output stream + * thrown if an I/O error occurs while writing the header data to the output stream */ - private void sendPutHeader(OutputStream outputStream, JobID jobID) throws IOException { - checkArgument(jobID == null); - + private static void sendPutHeader(OutputStream outputStream, @Nullable JobID jobId) throws IOException { // Signal type of operation outputStream.write(PUT_OPERATION); - - // Check if PUT should be done in content-addressable manner - outputStream.write(CONTENT_ADDRESSABLE); + if (jobId == null) { + outputStream.write(CONTENT_NO_JOB); + } else { + outputStream.write(CONTENT_FOR_JOB); + outputStream.write(jobId.getBytes()); + } } // -------------------------------------------------------------------------------------------- @@ -437,16 +531,50 @@ private void sendPutHeader(OutputStream outputStream, JobID jobID) throws IOExce // -------------------------------------------------------------------------------------------- /** - * Deletes the BLOB identified by the given BLOB key from the BLOB server. + * Deletes the (job-unrelated) BLOB identified by the given BLOB key from the BLOB server. + * + * @param key + * the key to identify the BLOB * - * @param blobKey - * the key to identify the BLOB * @throws IOException - * thrown if an I/O error occurs while transferring the request to - * the BLOB server or if the BLOB server cannot delete the file + * thrown if an I/O error occurs while transferring the request to the BLOB server or if the + * BLOB server cannot delete the file */ - public void delete(BlobKey blobKey) throws IOException { - checkArgument(blobKey != null, "BLOB key must not be null."); + public void delete(BlobKey key) throws IOException { + deleteInternal(null, key); + } + + /** + * Deletes the BLOB identified by the given BLOB key and job ID from the BLOB server. + * + * @param jobId + * the ID of job the BLOB belongs to + * @param key + * the key to identify the BLOB + * + * @throws IOException + * thrown if an I/O error occurs while transferring the request to the BLOB server or if the + * BLOB server cannot delete the file + */ + public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException { + checkNotNull(jobId); + deleteInternal(jobId, key); + } + + /** + * Deletes the BLOB identified by the given BLOB key and job ID from the BLOB server. + * + * @param jobId + * the ID of job the BLOB belongs to (or null if job-unrelated) + * @param key + * the key to identify the BLOB + * + * @throws IOException + * thrown if an I/O error occurs while transferring the request to the BLOB server or if the + * BLOB server cannot delete the file + */ + public void deleteInternal(@Nullable JobID jobId, BlobKey key) throws IOException { + checkNotNull(key); try { final OutputStream outputStream = this.socket.getOutputStream(); @@ -456,20 +584,16 @@ public void delete(BlobKey blobKey) throws IOException { outputStream.write(DELETE_OPERATION); // delete blob key - outputStream.write(CONTENT_ADDRESSABLE); - blobKey.writeToOutputStream(outputStream); - - int response = inputStream.read(); - if (response < 0) { - throw new EOFException("Premature end of response"); - } - if (response == RETURN_ERROR) { - Throwable cause = readExceptionFromStream(inputStream); - throw new IOException("Server side error: " + cause.getMessage(), cause); - } - else if (response != RETURN_OKAY) { - throw new IOException("Unrecognized response"); + if (jobId == null) { + outputStream.write(CONTENT_NO_JOB); + } else { + outputStream.write(CONTENT_FOR_JOB); + outputStream.write(jobId.getBytes()); } + key.writeToOutputStream(outputStream); + + // the response is the same as for a GET request + receiveAndCheckGetResponse(inputStream); } catch (Throwable t) { BlobUtils.closeSilently(socket, LOG); @@ -479,11 +603,18 @@ else if (response != RETURN_OKAY) { /** * Uploads the JAR files to a {@link BlobServer} at the given address. + *

+ * TODO: add jobId to signature after adapting the BlobLibraryCacheManager + * + * @param serverAddress + * Server address of the {@link BlobServer} + * @param clientConfig + * Any additional configuration for the blob client + * @param jars + * List of JAR files to upload * - * @param serverAddress Server address of the {@link BlobServer} - * @param clientConfig Any additional configuration for the blob client - * @param jars List of JAR files to upload - * @throws IOException Thrown if the upload fails + * @throws IOException + * if the upload fails */ public static List uploadJarFiles( InetSocketAddress serverAddress, @@ -500,7 +631,7 @@ public static List uploadJarFiles( FSDataInputStream is = null; try { is = fs.open(jar); - final BlobKey key = blobClient.put(is); + final BlobKey key = blobClient.putInputStream(null, is); blobKeys.add(key); } finally { if (is != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java index add9f7f7cdd4a..43a060a96f97a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; @@ -28,6 +29,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import javax.net.ssl.SSLContext; import java.io.File; import java.io.FileNotFoundException; @@ -40,7 +43,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -58,7 +61,7 @@ public class BlobServer extends Thread implements BlobService { private static final Logger LOG = LoggerFactory.getLogger(BlobServer.class); /** Counter to generate unique names for temporary files. */ - private final AtomicInteger tempFileCounter = new AtomicInteger(0); + private final AtomicLong tempFileCounter = new AtomicLong(0); /** The server socket listening for incoming connections. */ private final ServerSocket serverSocket; @@ -110,7 +113,7 @@ public BlobServer(Configuration config, BlobStore blobStore) throws IOException // configure and create the storage directory String storageDirectory = config.getString(BlobServerOptions.STORAGE_DIRECTORY); - this.storageDir = BlobUtils.initStorageDirectory(storageDirectory); + this.storageDir = BlobUtils.initLocalStorageDirectory(storageDirectory); LOG.info("Created BLOB server storage directory {}", storageDir); // configure the maximum number of concurrent connections @@ -189,11 +192,12 @@ public ServerSocket createSocket(int port) throws IOException { * *

This is only called from the {@link BlobServerConnection} * + * @param jobId ID of the job this blob belongs to (or null if job-unrelated) * @param key identifying the file * @return file handle to the file */ - File getStorageLocation(BlobKey key) { - return BlobUtils.getStorageLocation(storageDir, key); + File getStorageLocation(JobID jobId, BlobKey key) { + return BlobUtils.getStorageLocation(storageDir, jobId, key); } /** @@ -333,20 +337,69 @@ public BlobClient createClient() throws IOException { } /** - * Method which retrieves the local path of a file associated with a blob key. The blob server - * looks the blob key up in its local storage. If the file exists, it is returned. If the - * file does not exist, it is retrieved from the HA blob store (if available) or a - * FileNotFoundException is thrown. + * Retrieves the local path of a (job-unrelated) file associated with a job and a blob key. + *

+ * The blob server looks the blob key up in its local storage. If the file exists, it is + * returned. If the file does not exist, it is retrieved from the HA blob store (if available) + * or a {@link FileNotFoundException} is thrown. * - * @param requiredBlob blob key associated with the requested file - * @return file referring to the local storage location of the BLOB. - * @throws IOException Thrown if the file retrieval failed. + * @param key + * blob key associated with the requested file + * + * @return file referring to the local storage location of the BLOB + * + * @throws IOException + * Thrown if the file retrieval failed. + */ + @Override + public File getFile(BlobKey key) throws IOException { + return getFileInternal(null, key); + } + + /** + * Retrieves the local path of a file associated with a job and a blob key. + *

+ * The blob server looks the blob key up in its local storage. If the file exists, it is + * returned. If the file does not exist, it is retrieved from the HA blob store (if available) + * or a {@link FileNotFoundException} is thrown. + * + * @param jobId + * ID of the job this blob belongs to + * @param key + * blob key associated with the requested file + * + * @return file referring to the local storage location of the BLOB + * + * @throws IOException + * Thrown if the file retrieval failed. */ @Override - public File getFile(BlobKey requiredBlob) throws IOException { + public File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException { + checkNotNull(jobId); + return getFileInternal(jobId, key); + } + + /** + * Retrieves the local path of a file associated with a job and a blob key. + *

+ * The blob server looks the blob key up in its local storage. If the file exists, it is + * returned. If the file does not exist, it is retrieved from the HA blob store (if available) + * or a {@link FileNotFoundException} is thrown. + * + * @param jobId + * ID of the job this blob belongs to (or null if job-unrelated) + * @param requiredBlob + * blob key associated with the requested file + * + * @return file referring to the local storage location of the BLOB + * + * @throws IOException + * Thrown if the file retrieval failed. + */ + private File getFileInternal(@Nullable JobID jobId, BlobKey requiredBlob) throws IOException { checkArgument(requiredBlob != null, "BLOB key cannot be null."); - final File localFile = BlobUtils.getStorageLocation(storageDir, requiredBlob); + final File localFile = BlobUtils.getStorageLocation(storageDir, jobId, requiredBlob); if (localFile.exists()) { return localFile; @@ -354,10 +407,11 @@ public File getFile(BlobKey requiredBlob) throws IOException { else { try { // Try the blob store - blobStore.get(requiredBlob, localFile); + blobStore.get(jobId, requiredBlob, localFile); } catch (Exception e) { - throw new IOException("Failed to copy from blob store.", e); + throw new IOException( + "Failed to copy BLOB " + requiredBlob + " from blob store to " + localFile, e); } if (localFile.exists()) { @@ -371,24 +425,58 @@ public File getFile(BlobKey requiredBlob) throws IOException { } /** - * This method deletes the file associated to the blob key if it exists in the local storage - * of the blob server. + * Deletes the (job-unrelated) file associated with the blob key in both the local storage as + * well as in the HA store of the blob server. + * + * @param key + * blob key associated with the file to be deleted * - * @param key associated with the file to be deleted * @throws IOException */ @Override public void delete(BlobKey key) throws IOException { - final File localFile = BlobUtils.getStorageLocation(storageDir, key); + deleteInternal(null, key); + } + + /** + * Deletes the file associated with the blob key in both the local storage as well as in the HA + * store of the blob server. + * + * @param jobId + * ID of the job this blob belongs to + * @param key + * blob key associated with the file to be deleted + * + * @throws IOException + */ + @Override + public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException { + checkNotNull(jobId); + deleteInternal(jobId, key); + } + + /** + * Deletes the file associated with the blob key in both the local storage as well as in the HA + * store of the blob server. + * + * @param jobId + * ID of the job this blob belongs to (or null if job-unrelated) + * @param key + * blob key associated with the file to be deleted + * + * @throws IOException + */ + void deleteInternal(@Nullable JobID jobId, BlobKey key) throws IOException { + final File localFile = BlobUtils.getStorageLocation(storageDir, jobId, key); readWriteLock.writeLock().lock(); try { if (!localFile.delete() && localFile.exists()) { - LOG.warn("Failed to delete locally BLOB " + key + " at " + localFile.getAbsolutePath()); + LOG.warn("Failed to locally delete BLOB " + key + " at " + localFile.getAbsolutePath()); } - blobStore.delete(key); + blobStore.delete(jobId, key); } finally { readWriteLock.writeLock().unlock(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java index 181211d74eabe..f1054c090f1c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java @@ -39,7 +39,8 @@ import java.util.concurrent.locks.ReadWriteLock; import static org.apache.flink.runtime.blob.BlobServerProtocol.BUFFER_SIZE; -import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_ADDRESSABLE; +import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_FOR_JOB; +import static org.apache.flink.runtime.blob.BlobServerProtocol.CONTENT_NO_JOB; import static org.apache.flink.runtime.blob.BlobServerProtocol.DELETE_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.GET_OPERATION; import static org.apache.flink.runtime.blob.BlobServerProtocol.PUT_OPERATION; @@ -49,6 +50,7 @@ import static org.apache.flink.runtime.blob.BlobUtils.readFully; import static org.apache.flink.runtime.blob.BlobUtils.readLength; import static org.apache.flink.runtime.blob.BlobUtils.writeLength; +import static org.apache.flink.util.Preconditions.checkNotNull; /** * A BLOB connection handles a series of requests from a particular BLOB client. @@ -83,12 +85,8 @@ class BlobServerConnection extends Thread { super("BLOB connection for " + clientSocket.getRemoteSocketAddress()); setDaemon(true); - if (blobServer == null) { - throw new NullPointerException(); - } - this.clientSocket = clientSocket; - this.blobServer = blobServer; + this.blobServer = checkNotNull(blobServer); this.blobStore = blobServer.getBlobStore(); ReadWriteLock readWriteLock = blobServer.getReadWriteLock(); @@ -167,15 +165,16 @@ public void close() { /** * Handles an incoming GET request from a BLOB client. - * + * * @param inputStream - * the input stream to read incoming data from + * the input stream to read incoming data from * @param outputStream - * the output stream to send data back to the client + * the output stream to send data back to the client * @param buf - * an auxiliary buffer for data serialization/deserialization + * an auxiliary buffer for data serialization/deserialization + * * @throws IOException - * thrown if an I/O error occurs while reading/writing data from/to the respective streams + * thrown if an I/O error occurs while reading/writing data from/to the respective streams */ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) throws IOException { /* @@ -187,25 +186,36 @@ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) * so a local cache makes more sense. */ - File blobFile; - int contentAddressable = -1; - JobID jobId = null; - BlobKey blobKey = null; + final File blobFile; + final JobID jobId; + final BlobKey blobKey; try { - contentAddressable = inputStream.read(); + final int mode = inputStream.read(); - if (contentAddressable < 0) { + if (mode < 0) { throw new EOFException("Premature end of GET request"); } - if (contentAddressable == CONTENT_ADDRESSABLE) { - blobKey = BlobKey.readFromInputStream(inputStream); - blobFile = blobServer.getStorageLocation(blobKey); + + // Receive the job ID and key + if (mode == CONTENT_NO_JOB) { + jobId = null; + } else if (mode == CONTENT_FOR_JOB) { + byte[] jidBytes = new byte[JobID.SIZE]; + readFully(inputStream, jidBytes, 0, JobID.SIZE, "JobID"); + jobId = JobID.fromByteArray(jidBytes); + } else { + throw new IOException("Unknown type of BLOB addressing: " + mode + '.'); } - else { - throw new IOException("Unknown type of BLOB addressing: " + contentAddressable + '.'); + blobKey = BlobKey.readFromInputStream(inputStream); + + if (LOG.isDebugEnabled()) { + LOG.debug("Received GET request for BLOB {}/{} from {}.", jobId, + blobKey, clientSocket.getInetAddress()); } + blobFile = blobServer.getStorageLocation(jobId, blobKey); + // up to here, an error can give a good message } catch (Throwable t) { @@ -214,7 +224,7 @@ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) writeErrorToStream(outputStream, t); } catch (IOException e) { - // since we are in an exception case, it means not much that we could not send the error + // since we are in an exception case, it means that we could not send the error // ignore this } clientSocket.close(); @@ -224,6 +234,7 @@ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) readLock.lock(); try { + // copy the file to local store if it does not exist yet try { if (!blobFile.exists()) { // first we have to release the read lock in order to acquire the write lock @@ -232,9 +243,9 @@ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) try { if (blobFile.exists()) { - LOG.debug("Blob file {} has downloaded from the BlobStore by a different connection.", blobFile); + LOG.debug("Blob file {} has been downloaded from the (distributed) blob store by a different connection.", blobFile); } else { - blobStore.get(blobKey, blobFile); + blobStore.get(jobId, blobKey, blobFile); } } finally { writeLock.unlock(); @@ -248,6 +259,7 @@ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) } } + // enforce a 2GB max for now (otherwise the protocol's length field needs to be increased) if (blobFile.length() > Integer.MAX_VALUE) { throw new IOException("BLOB size exceeds the maximum size (2 GB)."); } @@ -259,7 +271,7 @@ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) writeErrorToStream(outputStream, t); } catch (IOException e) { - // since we are in an exception case, it means not much that we could not send the error + // since we are in an exception case, it means that we could not send the error // ignore this } clientSocket.close(); @@ -294,59 +306,48 @@ private void get(InputStream inputStream, OutputStream outputStream, byte[] buf) /** * Handles an incoming PUT request from a BLOB client. - * - * @param inputStream The input stream to read incoming data from. - * @param outputStream The output stream to send data back to the client. - * @param buf An auxiliary buffer for data serialization/deserialization. + * + * @param inputStream + * The input stream to read incoming data from + * @param outputStream + * The output stream to send data back to the client + * @param buf + * An auxiliary buffer for data serialization/deserialization + * + * @throws IOException + * thrown if an I/O error occurs while reading/writing data from/to the respective streams */ private void put(InputStream inputStream, OutputStream outputStream, byte[] buf) throws IOException { - JobID jobID = null; - MessageDigest md = null; - File incomingFile = null; - FileOutputStream fos = null; try { - final int contentAddressable = inputStream.read(); - if (contentAddressable < 0) { + final int mode = inputStream.read(); + + if (mode < 0) { throw new EOFException("Premature end of PUT request"); } - if (contentAddressable == CONTENT_ADDRESSABLE) { - md = BlobUtils.createMessageDigest(); - } - else { + // Receive the job ID and key + final JobID jobId; + if (mode == CONTENT_NO_JOB) { + jobId = null; + } else if (mode == CONTENT_FOR_JOB) { + byte[] jidBytes = new byte[JobID.SIZE]; + readFully(inputStream, jidBytes, 0, JobID.SIZE, "JobID"); + jobId = JobID.fromByteArray(jidBytes); + } else { throw new IOException("Unknown type of BLOB addressing."); } if (LOG.isDebugEnabled()) { - LOG.debug("Received PUT request for content addressable BLOB"); + LOG.debug("Received PUT request for BLOB of job {} with from {}.", jobId, + clientSocket.getInetAddress()); } incomingFile = blobServer.createTemporaryFilename(); - fos = new FileOutputStream(incomingFile); + BlobKey blobKey = readFileFully(inputStream, incomingFile, buf); - while (true) { - final int bytesExpected = readLength(inputStream); - if (bytesExpected == -1) { - // done - break; - } - if (bytesExpected > BUFFER_SIZE) { - throw new IOException("Unexpected number of incoming bytes: " + bytesExpected); - } - - readFully(inputStream, buf, 0, bytesExpected, "buffer"); - fos.write(buf, 0, bytesExpected); - - if (md != null) { - md.update(buf, 0, bytesExpected); - } - } - fos.close(); - - BlobKey blobKey = new BlobKey(md.digest()); - File storageFile = blobServer.getStorageLocation(blobKey); + File storageFile = blobServer.getStorageLocation(jobId, blobKey); writeLock.lock(); @@ -369,13 +370,15 @@ private void put(InputStream inputStream, OutputStream outputStream, byte[] buf) // only the one moving the incoming file to its final destination is allowed to upload the // file to the blob store - blobStore.put(storageFile, blobKey); + blobStore.put(storageFile, jobId, blobKey); + } else { + LOG.warn("File upload for an existing file with key {} for job {}. This may indicate a duplicate upload or a hash collision. Ignoring newest upload.", blobKey, jobId); } } catch(IOException ioe) { // we failed to either create the local storage file or to upload it --> try to delete the local file // while still having the write lock - if (storageFile.exists() && !storageFile.delete()) { - LOG.warn("Could not delete the storage file."); + if (!storageFile.delete() && storageFile.exists()) { + LOG.warn("Could not delete the storage file with key {} and job {}.", blobKey, jobId); } throw ioe; @@ -403,43 +406,96 @@ private void put(InputStream inputStream, OutputStream outputStream, byte[] buf) clientSocket.close(); } finally { - if (fos != null) { - try { - fos.close(); - } catch (Throwable t) { - LOG.warn("Cannot close stream to BLOB staging file", t); - } - } if (incomingFile != null) { - if (!incomingFile.delete()) { + if (!incomingFile.delete() && incomingFile.exists()) { LOG.warn("Cannot delete BLOB server staging file " + incomingFile.getAbsolutePath()); } } } } + /** + * Reads a full file from inputStream into incomingFile returning its checksum. + * + * @param inputStream + * stream to read from + * @param incomingFile + * file to write to + * @param buf + * An auxiliary buffer for data serialization/deserialization + * + * @return the received file's content hash as a BLOB key + * + * @throws IOException + * thrown if an I/O error occurs while reading/writing data from/to the respective streams + */ + private static BlobKey readFileFully( + final InputStream inputStream, final File incomingFile, final byte[] buf) + throws IOException { + MessageDigest md = BlobUtils.createMessageDigest(); + FileOutputStream fos = new FileOutputStream(incomingFile); + + try { + while (true) { + final int bytesExpected = readLength(inputStream); + if (bytesExpected == -1) { + // done + break; + } + if (bytesExpected > BUFFER_SIZE) { + throw new IOException( + "Unexpected number of incoming bytes: " + bytesExpected); + } + + readFully(inputStream, buf, 0, bytesExpected, "buffer"); + fos.write(buf, 0, bytesExpected); + + md.update(buf, 0, bytesExpected); + } + return new BlobKey(md.digest()); + } finally { + try { + fos.close(); + } catch (Throwable t) { + LOG.warn("Cannot close stream to BLOB staging file", t); + } + } + } + /** * Handles an incoming DELETE request from a BLOB client. - * - * @param inputStream The input stream to read the request from. - * @param outputStream The output stream to write the response to. - * @throws java.io.IOException Thrown if an I/O error occurs while reading the request data from the input stream. + * + * @param inputStream + * The input stream to read the request from. + * @param outputStream + * The output stream to write the response to. + * + * @throws IOException + * Thrown if an I/O error occurs while reading the request data from the input stream. */ private void delete(InputStream inputStream, OutputStream outputStream) throws IOException { try { - int type = inputStream.read(); - if (type < 0) { + final int mode = inputStream.read(); + + if (mode < 0) { throw new EOFException("Premature end of DELETE request"); } - if (type == CONTENT_ADDRESSABLE) { - BlobKey key = BlobKey.readFromInputStream(inputStream); - blobServer.delete(key); - } - else { - throw new IOException("Unrecognized addressing type: " + type); + // Receive the job ID and key + final JobID jobId; + if (mode == CONTENT_NO_JOB) { + jobId = null; + } else if (mode == CONTENT_FOR_JOB) { + byte[] jidBytes = new byte[JobID.SIZE]; + readFully(inputStream, jidBytes, 0, JobID.SIZE, "JobID"); + jobId = JobID.fromByteArray(jidBytes); + } else { + throw new IOException("Unknown type of BLOB addressing."); } + BlobKey key = BlobKey.readFromInputStream(inputStream); + + blobServer.deleteInternal(jobId, key); outputStream.write(RETURN_OKAY); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java index d8ac83373b573..681fc8150be3f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerProtocol.java @@ -42,12 +42,20 @@ public class BlobServerProtocol { static final byte RETURN_ERROR = 1; /** - * Internal code to identify a reference via content hash as the key. + * Internal code to identify a job-unrelated reference via content hash as the key. *

* Note: previously, there was also NAME_ADDRESSABLE (code 1) and * JOB_ID_SCOPE (code 2). */ - static final byte CONTENT_ADDRESSABLE = 0; + static final byte CONTENT_NO_JOB = 0; + + /** + * Internal code to identify a job-related reference via content hash as the key. + *

+ * Note: previously, there was also NAME_ADDRESSABLE (code 1) and + * JOB_ID_SCOPE (code 2). + */ + static final byte CONTENT_FOR_JOB = 3; // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java index 1e56f26cabf3e..a78c88c8e89b3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java @@ -18,6 +18,9 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.api.common.JobID; + +import javax.annotation.Nonnull; import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -28,7 +31,8 @@ public interface BlobService extends Closeable { /** - * Returns the path to a local copy of the file associated with the provided blob key. + * Returns the path to a local copy of the (job-unrelated) file associated with the provided + * blob key. * * @param key blob key associated with the requested file * @return The path to the file. @@ -37,20 +41,39 @@ public interface BlobService extends Closeable { */ File getFile(BlobKey key) throws IOException; + /** + * Returns the path to a local copy of the file associated with the provided job ID and blob key. + * + * @param jobId ID of the job this blob belongs to + * @param key blob key associated with the requested file + * @return The path to the file. + * @throws java.io.FileNotFoundException when the path does not exist; + * @throws IOException if any other error occurs when retrieving the file + */ + File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException; /** - * Deletes the file associated with the provided blob key. + * Deletes the (job-unrelated) file associated with the provided blob key. * * @param key associated with the file to be deleted * @throws IOException */ void delete(BlobKey key) throws IOException; + /** + * Deletes the file associated with the provided job ID and blob key. + * + * @param jobId ID of the job this blob belongs to + * @param key associated with the file to be deleted + * @throws IOException + */ + void delete(@Nonnull JobID jobId, BlobKey key) throws IOException; + /** * Returns the port of the blob service. * @return the port of the blob service. */ int getPort(); - + BlobClient createClient() throws IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java index 1e8b73a43a1d8..d2ea8caaf2616 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobStore.java @@ -32,19 +32,21 @@ public interface BlobStore extends BlobView { * Copies the local file to the blob store. * * @param localFile The file to copy + * @param jobId ID of the job this blob belongs to (or null if job-unrelated) * @param blobKey The ID for the file in the blob store * @throws IOException If the copy fails */ - void put(File localFile, BlobKey blobKey) throws IOException; + void put(File localFile, JobID jobId, BlobKey blobKey) throws IOException; /** * Tries to delete a blob from storage. * *

NOTE: This also tries to delete any created directories if empty.

* + * @param jobId ID of the job this blob belongs to (or null if job-unrelated) * @param blobKey The blob ID */ - void delete(BlobKey blobKey); + void delete(JobID jobId, BlobKey blobKey); /** * Tries to delete all blobs for the given job from storage. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java index e8f3fe575e187..9a13412c90b84 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java @@ -28,6 +28,8 @@ import org.apache.flink.util.StringUtils; import org.slf4j.Logger; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.EOFException; import java.io.File; import java.io.IOException; @@ -61,6 +63,11 @@ public class BlobUtils { */ private static final String JOB_DIR_PREFIX = "job_"; + /** + * The prefix of all job-unrelated directories created by the BLOB server. + */ + private static final String NO_JOB_DIR_PREFIX = "no_job"; + /** * Creates a BlobStore based on the parameters set in the configuration. * @@ -116,26 +123,29 @@ private static BlobStoreService createFileSystemBlobStore(Configuration configur } /** - * Creates a storage directory for a blob service. + * Creates a local storage directory for a blob service under the given parent directory. * - * @return the storage directory used by a BLOB service + * @param basePath + * base path, i.e. parent directory, of the storage directory to use (if null or + * empty, the path in java.io.tmpdir will be used) + * + * @return a new local storage directory * * @throws IOException - * thrown if the (local or distributed) file storage cannot be created or - * is not usable + * thrown if the local file storage cannot be created or is not usable */ - static File initStorageDirectory(String storageDirectory) throws - IOException { + static File initLocalStorageDirectory(String basePath) throws IOException { File baseDir; - if (StringUtils.isNullOrWhitespaceOnly(storageDirectory)) { + if (StringUtils.isNullOrWhitespaceOnly(basePath)) { baseDir = new File(System.getProperty("java.io.tmpdir")); } else { - baseDir = new File(storageDirectory); + baseDir = new File(basePath); } File storageDir; + // NOTE: although we will be using UUIDs, there may be collisions final int MAX_ATTEMPTS = 10; for(int attempt = 0; attempt < MAX_ATTEMPTS; attempt++) { storageDir = new File(baseDir, String.format( @@ -143,7 +153,7 @@ static File initStorageDirectory(String storageDirectory) throws // Create the storage dir if it doesn't exist. Only return it when the operation was // successful. - if (!storageDir.exists() && storageDir.mkdirs()) { + if (storageDir.mkdirs()) { return storageDir; } } @@ -153,46 +163,108 @@ static File initStorageDirectory(String storageDirectory) throws } /** - * Returns the BLOB service's directory for incoming files. The directory is created if it did - * not exist so far. + * Returns the BLOB service's directory for incoming (job-unrelated) files. The directory is + * created if it does not exist yet. + * + * @param storageDir + * storage directory used be the BLOB service * - * @return the BLOB server's directory for incoming files + * @return the BLOB service's directory for incoming files */ static File getIncomingDirectory(File storageDir) { final File incomingDir = new File(storageDir, "incoming"); - if (!incomingDir.mkdirs() && !incomingDir.exists()) { - throw new RuntimeException("Cannot create directory for incoming files " + incomingDir.getAbsolutePath()); - } + mkdirTolerateExisting(incomingDir, "incoming"); return incomingDir; } /** - * Returns the BLOB service's directory for cached files. The directory is created if it did - * not exist so far. + * Makes sure a given directory exists by creating it if necessary. * - * @return the BLOB server's directory for cached files + * @param dir + * directory to create + * @param dirType + * the type of the directory (included in error message if something fails) */ - private static File getCacheDirectory(File storageDir) { - final File cacheDirectory = new File(storageDir, "cache"); - - if (!cacheDirectory.mkdirs() && !cacheDirectory.exists()) { - throw new RuntimeException("Could not create cache directory '" + cacheDirectory.getAbsolutePath() + "'."); + private static void mkdirTolerateExisting(final File dir, final String dirType) { + // note: thread-safe create should try to mkdir first and then ignore the case that the + // directory already existed + if (!dir.mkdirs() && !dir.exists()) { + throw new RuntimeException( + "Cannot create " + dirType + " directory '" + dir.getAbsolutePath() + "'."); } - - return cacheDirectory; } /** * Returns the (designated) physical storage location of the BLOB with the given key. * + * @param storageDir + * storage directory used be the BLOB service * @param key - * the key identifying the BLOB + * the key identifying the BLOB + * @param jobId + * ID of the job for the incoming files (or null if job-unrelated) + * * @return the (designated) physical storage location of the BLOB */ - static File getStorageLocation(File storageDir, BlobKey key) { - return new File(getCacheDirectory(storageDir), BLOB_FILE_PREFIX + key.toString()); + static File getStorageLocation( + @Nonnull File storageDir, @Nullable JobID jobId, @Nonnull BlobKey key) { + File file = new File(getStorageLocationPath(storageDir.getAbsolutePath(), jobId, key)); + + mkdirTolerateExisting(file.getParentFile(), "cache"); + + return file; + } + + /** + * Returns the BLOB server's storage directory for BLOBs belonging to the job with the given ID + * without creating the directory. + * + * @param storageDir + * storage directory used be the BLOB service + * @param jobId + * the ID of the job to return the storage directory for + * + * @return the storage directory for BLOBs belonging to the job with the given ID + */ + static String getStorageLocationPath(@Nonnull String storageDir, @Nullable JobID jobId) { + if (jobId == null) { + // format: $base/no_job + return String.format("%s/%s", storageDir, NO_JOB_DIR_PREFIX); + } else { + // format: $base/job_$jobId + return String.format("%s/%s%s", storageDir, JOB_DIR_PREFIX, jobId.toString()); + } + } + + /** + * Returns the path for the given blob key. + *

+ * The returned path can be used with the (local or HA) BLOB store file system back-end for + * recovery purposes and follows the same scheme as {@link #getStorageLocation(File, JobID, + * BlobKey)}. + * + * @param storageDir + * storage directory used be the BLOB service + * @param key + * the key identifying the BLOB + * @param jobId + * ID of the job for the incoming files + * + * @return the path to the given BLOB + */ + static String getStorageLocationPath( + @Nonnull String storageDir, @Nullable JobID jobId, @Nonnull BlobKey key) { + if (jobId == null) { + // format: $base/no_job/blob_$key + return String.format("%s/%s/%s%s", + storageDir, NO_JOB_DIR_PREFIX, BLOB_FILE_PREFIX, key.toString()); + } else { + // format: $base/job_$jobId/blob_$key + return String.format("%s/%s%s/%s%s", + storageDir, JOB_DIR_PREFIX, jobId.toString(), BLOB_FILE_PREFIX, key.toString()); + } } /** @@ -200,6 +272,7 @@ static File getStorageLocation(File storageDir, BlobKey key) { * * @return a new instance of the message digest to use for the BLOB key computation */ + @Nonnull static MessageDigest createMessageDigest() { try { return MessageDigest.getInstance(HASHING_ALGORITHM); @@ -332,28 +405,6 @@ static void closeSilently(Socket socket, Logger LOG) { } } - /** - * Returns the path for the given blob key. - * - *

The returned path can be used with the state backend for recovery purposes. - * - *

This follows the same scheme as {@link #getStorageLocation(File, BlobKey)} - * and is used for HA. - */ - static String getRecoveryPath(String basePath, BlobKey blobKey) { - // format: $base/cache/blob_$key - return String.format("%s/cache/%s%s", basePath, BLOB_FILE_PREFIX, blobKey.toString()); - } - - /** - * Returns the path for the given job ID. - * - *

The returned path can be used with the state backend for recovery purposes. - */ - static String getRecoveryPath(String basePath, JobID jobId) { - return String.format("%s/%s%s", basePath, JOB_DIR_PREFIX, jobId.toString()); - } - /** * Private constructor to prevent instantiation. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java index 2e2e4a77841b6..8916d953d7eff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobView.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.api.common.JobID; + import java.io.File; import java.io.IOException; @@ -29,9 +31,10 @@ public interface BlobView { /** * Copies a blob to a local file. * + * @param jobId ID of the job this blob belongs to (or null if job-unrelated) * @param blobKey The blob ID * @param localFile The local file to copy to * @throws IOException If the copy fails */ - void get(BlobKey blobKey, File localFile) throws IOException; + void get(JobID jobId, BlobKey blobKey, File localFile) throws IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java index 5f8058bee5e77..83abecb478b16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java @@ -64,8 +64,8 @@ public FileSystemBlobStore(FileSystem fileSystem, String storagePath) throws IOE // - Put ------------------------------------------------------------------ @Override - public void put(File localFile, BlobKey blobKey) throws IOException { - put(localFile, BlobUtils.getRecoveryPath(basePath, blobKey)); + public void put(File localFile, JobID jobId, BlobKey blobKey) throws IOException { + put(localFile, BlobUtils.getStorageLocationPath(basePath, jobId, blobKey)); } private void put(File fromFile, String toBlobPath) throws IOException { @@ -78,8 +78,8 @@ private void put(File fromFile, String toBlobPath) throws IOException { // - Get ------------------------------------------------------------------ @Override - public void get(BlobKey blobKey, File localFile) throws IOException { - get(BlobUtils.getRecoveryPath(basePath, blobKey), localFile); + public void get(JobID jobId, BlobKey blobKey, File localFile) throws IOException { + get(BlobUtils.getStorageLocationPath(basePath, jobId, blobKey), localFile); } private void get(String fromBlobPath, File toFile) throws IOException { @@ -112,13 +112,13 @@ private void get(String fromBlobPath, File toFile) throws IOException { // - Delete --------------------------------------------------------------- @Override - public void delete(BlobKey blobKey) { - delete(BlobUtils.getRecoveryPath(basePath, blobKey)); + public void delete(JobID jobId, BlobKey blobKey) { + delete(BlobUtils.getStorageLocationPath(basePath, jobId, blobKey)); } @Override public void deleteAll(JobID jobId) { - delete(BlobUtils.getRecoveryPath(basePath, jobId)); + delete(BlobUtils.getStorageLocationPath(basePath, jobId)); } private void delete(String blobPath) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java index 6e2bb53ff08d7..95be5697920db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/VoidBlobStore.java @@ -29,16 +29,15 @@ public class VoidBlobStore implements BlobStoreService { @Override - public void put(File localFile, BlobKey blobKey) throws IOException { + public void put(File localFile, JobID jobId, BlobKey blobKey) throws IOException { } - @Override - public void get(BlobKey blobKey, File localFile) throws IOException { + public void get(JobID jobId, BlobKey blobKey, File localFile) throws IOException { } @Override - public void delete(BlobKey blobKey) { + public void delete(JobID jobId, BlobKey blobKey) { } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java index e3657ff54145e..9cc621094570a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java @@ -234,6 +234,7 @@ public static ClassLoader retrieveClassLoader( int pos = 0; for (BlobKey blobKey : props.requiredJarFiles()) { try { + // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager allURLs[pos++] = blobClient.getFile(blobKey).toURI().toURL(); } catch (Exception e) { try { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java index 1c68515f83965..6b92d796d8709 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java @@ -535,6 +535,7 @@ public void uploadUserJars( InetSocketAddress blobServerAddress, Configuration blobClientConfig) throws IOException { if (!userJars.isEmpty()) { + // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager List blobKeys = BlobClient.uploadJarFiles(blobServerAddress, blobClientConfig, userJars); for (BlobKey blobKey : blobKeys) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java index fe763fa20dad7..8c575a959b9e5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; @@ -25,6 +26,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.net.InetSocketAddress; @@ -41,7 +43,8 @@ public class BlobCacheRetriesTest { public TemporaryFolder temporaryFolder = new TemporaryFolder(); /** - * A test where the connection fails twice and then the get operation succeeds. + * A test where the connection fails twice and then the get operation succeeds + * (job-unrelated blob). */ @Test public void testBlobFetchRetries() throws IOException { @@ -49,15 +52,41 @@ public void testBlobFetchRetries() throws IOException { config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); - testBlobFetchRetries(config, new VoidBlobStore()); + testBlobFetchRetries(config, new VoidBlobStore(), null); + } + + /** + * A test where the connection fails twice and then the get operation succeeds + * (job-related blob). + */ + @Test + public void testBlobForJobFetchRetries() throws IOException { + final Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + + testBlobFetchRetries(config, new VoidBlobStore(), new JobID()); + } + + /** + * A test where the connection fails twice and then the get operation succeeds + * (with high availability set, job-unrelated blob). + */ + @Test + public void testBlobNoJobFetchRetriesHa() throws IOException { + testBlobFetchRetriesHa(null); } /** * A test where the connection fails twice and then the get operation succeeds - * (with high availability set). + * (with high availability set, job-related job). */ @Test public void testBlobFetchRetriesHa() throws IOException { + testBlobFetchRetriesHa(new JobID()); + } + + private void testBlobFetchRetriesHa(final JobID jobId) throws IOException { final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); @@ -70,7 +99,7 @@ public void testBlobFetchRetriesHa() throws IOException { try { blobStoreService = BlobUtils.createBlobStoreFromConfig(config); - testBlobFetchRetries(config, blobStoreService); + testBlobFetchRetries(config, blobStoreService, jobId); } finally { if (blobStoreService != null) { blobStoreService.closeAndCleanupAllData(); @@ -86,7 +115,9 @@ public void testBlobFetchRetriesHa() throws IOException { * configuration to use (the BlobCache will get some additional settings * set compared to this one) */ - private void testBlobFetchRetries(final Configuration config, final BlobStore blobStore) throws IOException { + private static void testBlobFetchRetries( + final Configuration config, final BlobStore blobStore, final JobID jobId) + throws IOException { final byte[] data = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 0}; BlobServer server = null; @@ -104,7 +135,7 @@ private void testBlobFetchRetries(final Configuration config, final BlobStore bl try { blobClient = new BlobClient(serverAddress, config); - key = blobClient.put(data); + key = blobClient.put(jobId, data); } finally { if (blobClient != null) { @@ -115,16 +146,13 @@ private void testBlobFetchRetries(final Configuration config, final BlobStore bl cache = new BlobCache(serverAddress, config, new VoidBlobStore()); // trigger a download - it should fail the first two times, but retry, and succeed eventually - URL url = cache.getFile(key).toURI().toURL(); - InputStream is = url.openStream(); - try { + File file = jobId == null ? cache.getFile(key) : cache.getFile(jobId, key); + URL url = file.toURI().toURL(); + try (InputStream is = url.openStream()) { byte[] received = new byte[data.length]; assertEquals(data.length, is.read(received)); assertArrayEquals(data, received); } - finally { - is.close(); - } } finally { if (cache != null) { cache.close(); @@ -136,23 +164,50 @@ private void testBlobFetchRetries(final Configuration config, final BlobStore bl } /** - * A test where the connection fails too often and eventually fails the GET request. + * A test where the connection fails too often and eventually fails the GET request + * (job-unrelated blob). + */ + @Test + public void testBlobNoJobFetchWithTooManyFailures() throws IOException { + final Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + + testBlobFetchWithTooManyFailures(config, new VoidBlobStore(), null); + } + + /** + * A test where the connection fails too often and eventually fails the GET request (job-related + * blob). */ @Test - public void testBlobFetchWithTooManyFailures() throws IOException { + public void testBlobForJobFetchWithTooManyFailures() throws IOException { final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); - testBlobFetchWithTooManyFailures(config, new VoidBlobStore()); + testBlobFetchWithTooManyFailures(config, new VoidBlobStore(), new JobID()); + } + + /** + * A test where the connection fails twice and then the get operation succeeds + * (with high availability set, job-unrelated blob). + */ + @Test + public void testBlobNoJobFetchWithTooManyFailuresHa() throws IOException { + testBlobFetchWithTooManyFailuresHa(null); } /** * A test where the connection fails twice and then the get operation succeeds - * (with high availability set). + * (with high availability set, job-related blob). */ @Test - public void testBlobFetchWithTooManyFailuresHa() throws IOException { + public void testBlobForJobFetchWithTooManyFailuresHa() throws IOException { + testBlobFetchWithTooManyFailuresHa(new JobID()); + } + + private void testBlobFetchWithTooManyFailuresHa(final JobID jobId) throws IOException { final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); @@ -165,7 +220,7 @@ public void testBlobFetchWithTooManyFailuresHa() throws IOException { try { blobStoreService = BlobUtils.createBlobStoreFromConfig(config); - testBlobFetchWithTooManyFailures(config, blobStoreService); + testBlobFetchWithTooManyFailures(config, blobStoreService, jobId); } finally { if (blobStoreService != null) { blobStoreService.closeAndCleanupAllData(); @@ -181,7 +236,9 @@ public void testBlobFetchWithTooManyFailuresHa() throws IOException { * configuration to use (the BlobCache will get some additional settings * set compared to this one) */ - private void testBlobFetchWithTooManyFailures(final Configuration config, final BlobStore blobStore) throws IOException { + private static void testBlobFetchWithTooManyFailures( + final Configuration config, final BlobStore blobStore, final JobID jobId) + throws IOException { final byte[] data = new byte[] { 1, 2, 3, 4, 5, 6, 7, 8, 9, 0 }; BlobServer server = null; @@ -199,7 +256,7 @@ private void testBlobFetchWithTooManyFailures(final Configuration config, final try { blobClient = new BlobClient(serverAddress, config); - key = blobClient.put(data); + key = blobClient.put(jobId, data); } finally { if (blobClient != null) { @@ -211,7 +268,11 @@ private void testBlobFetchWithTooManyFailures(final Configuration config, final // trigger a download - it should fail eventually try { - cache.getFile(key); + if (jobId == null) { + cache.getFile(key); + } else { + cache.getFile(jobId, key); + } fail("This should fail"); } catch (IOException e) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java index d06f76f167174..1216be2daf093 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheSuccessTest.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLogger; import org.junit.Rule; import org.junit.Test; @@ -40,69 +42,123 @@ /** * This class contains unit tests for the {@link BlobCache}. */ -public class BlobCacheSuccessTest { +public class BlobCacheSuccessTest extends TestLogger { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); /** - * BlobCache with no HA. BLOBs need to be downloaded form a working + * BlobCache with no HA, job-unrelated BLOBs. BLOBs need to be downloaded form a working * BlobServer. */ @Test - public void testBlobCache() throws IOException { + public void testBlobNoJobCache() throws IOException { Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); - uploadFileGetTest(config, false, false); + uploadFileGetTest(config, null, false, false); + } + + /** + * BlobCache with no HA, job-related BLOBS. BLOBs need to be downloaded form a working + * BlobServer. + */ + @Test + public void testBlobForJobCache() throws IOException { + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + + uploadFileGetTest(config, new JobID(), false, false); } /** * BlobCache is configured in HA mode and the cache can download files from * the file system directly and does not need to download BLOBs from the - * BlobServer. + * BlobServer. Using job-unrelated BLOBs. + */ + @Test + public void testBlobNoJobCacheHa() throws IOException { + testBlobCacheHa(null); + } + + /** + * BlobCache is configured in HA mode and the cache can download files from + * the file system directly and does not need to download BLOBs from the + * BlobServer. Using job-related BLOBs. */ @Test - public void testBlobCacheHa() throws IOException { + public void testBlobForJobCacheHa() throws IOException { + testBlobCacheHa(new JobID()); + } + + private void testBlobCacheHa(final JobID jobId) throws IOException { Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER"); config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getPath()); - uploadFileGetTest(config, true, true); + uploadFileGetTest(config, jobId, true, true); } /** * BlobCache is configured in HA mode and the cache can download files from * the file system directly and does not need to download BLOBs from the - * BlobServer. + * BlobServer. Using job-unrelated BLOBs. + */ + @Test + public void testBlobNoJobCacheHa2() throws IOException { + testBlobCacheHa2(null); + } + + /** + * BlobCache is configured in HA mode and the cache can download files from + * the file system directly and does not need to download BLOBs from the + * BlobServer. Using job-related BLOBs. */ @Test - public void testBlobCacheHa2() throws IOException { + public void testBlobForJobCacheHa2() throws IOException { + testBlobCacheHa2(new JobID()); + } + + private void testBlobCacheHa2(JobID jobId) throws IOException { Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER"); config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getPath()); - uploadFileGetTest(config, false, true); + uploadFileGetTest(config, jobId, false, true); } /** * BlobCache is configured in HA mode but the cache itself cannot access the - * file system and thus needs to download BLOBs from the BlobServer. + * file system and thus needs to download BLOBs from the BlobServer. Using job-unrelated BLOBs. */ @Test - public void testBlobCacheHaFallback() throws IOException { + public void testBlobNoJobCacheHaFallback() throws IOException { + testBlobCacheHaFallback(null); + } + + /** + * BlobCache is configured in HA mode but the cache itself cannot access the + * file system and thus needs to download BLOBs from the BlobServer. Using job-related BLOBs. + */ + @Test + public void testBlobForJobCacheHaFallback() throws IOException { + testBlobCacheHaFallback(new JobID()); + } + + private void testBlobCacheHaFallback(final JobID jobId) throws IOException { Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER"); config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getPath()); - uploadFileGetTest(config, false, false); + uploadFileGetTest(config, jobId, false, false); } /** @@ -119,7 +175,7 @@ public void testBlobCacheHaFallback() throws IOException { * whether the cache should have access to a shared HA_STORAGE_PATH (only useful with * HA mode) */ - private void uploadFileGetTest(final Configuration config, boolean shutdownServerAfterUpload, + private void uploadFileGetTest(final Configuration config, JobID jobId, boolean shutdownServerAfterUpload, boolean cacheHasAccessToFs) throws IOException { Preconditions.checkArgument(!shutdownServerAfterUpload || cacheHasAccessToFs); @@ -154,9 +210,9 @@ private void uploadFileGetTest(final Configuration config, boolean shutdownServe blobClient = new BlobClient(serverAddress, config); - blobKeys.add(blobClient.put(buf)); + blobKeys.add(blobClient.put(jobId, buf)); buf[0] = 1; // Make sure the BLOB key changes - blobKeys.add(blobClient.put(buf)); + blobKeys.add(blobClient.put(jobId, buf)); } finally { if (blobClient != null) { blobClient.close(); @@ -172,7 +228,11 @@ private void uploadFileGetTest(final Configuration config, boolean shutdownServe blobCache = new BlobCache(serverAddress, cacheConfig, blobStoreService); for (BlobKey blobKey : blobKeys) { - blobCache.getFile(blobKey); + if (jobId == null) { + blobCache.getFile(blobKey); + } else { + blobCache.getFile(jobId, blobKey); + } } if (blobServer != null) { @@ -184,7 +244,11 @@ private void uploadFileGetTest(final Configuration config, boolean shutdownServe final File[] files = new File[blobKeys.size()]; for(int i = 0; i < blobKeys.size(); i++){ - files[i] = blobCache.getFile(blobKeys.get(i)); + if (jobId == null) { + files[i] = blobCache.getFile(blobKeys.get(i)); + } else { + files[i] = blobCache.getFile(jobId, blobKeys.get(i)); + } } // Verify the result diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java index 2932f41daea2c..cfec4c5f51494 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java @@ -38,6 +38,7 @@ import java.util.Collections; import java.util.List; +import org.apache.flink.api.common.JobID; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -144,7 +145,7 @@ private static BlobKey prepareTestFile(File file) throws IOException { * @throws IOException * thrown if an I/O error occurs while reading the input stream */ - private static void validateGet(final InputStream inputStream, final byte[] buf) throws IOException { + static void validateGet(final InputStream inputStream, final byte[] buf) throws IOException { byte[] receivedBuffer = new byte[buf.length]; int bytesReceived = 0; @@ -220,13 +221,20 @@ public void testContentAddressableBuffer() { InetSocketAddress serverAddress = new InetSocketAddress("localhost", getBlobServer().getPort()); client = new BlobClient(serverAddress, getBlobClientConfig()); + JobID jobId = new JobID(); + // Store the data - BlobKey receivedKey = client.put(testBuffer); + BlobKey receivedKey = client.put(null, testBuffer); + assertEquals(origKey, receivedKey); + // try again with a job-related BLOB: + receivedKey = client.put(jobId, testBuffer); assertEquals(origKey, receivedKey); // Retrieve the data InputStream is = client.get(receivedKey); validateGet(is, testBuffer); + is = client.get(jobId, receivedKey); + validateGet(is, testBuffer); // Check reaction to invalid keys try { @@ -236,6 +244,15 @@ public void testContentAddressableBuffer() { catch (IOException fnfe) { // expected } + // new client needed (closed from failure above) + client = new BlobClient(serverAddress, getBlobClientConfig()); + try { + client.get(jobId, new BlobKey()); + fail("Expected IOException did not occur"); + } + catch (IOException fnfe) { + // expected + } } catch (Exception e) { e.printStackTrace(); @@ -276,10 +293,16 @@ public void testContentAddressableStream() { InetSocketAddress serverAddress = new InetSocketAddress("localhost", getBlobServer().getPort()); client = new BlobClient(serverAddress, getBlobClientConfig()); + JobID jobId = new JobID(); + // Store the data is = new FileInputStream(testFile); BlobKey receivedKey = client.put(is); assertEquals(origKey, receivedKey); + // try again with a job-related BLOB: + is = new FileInputStream(testFile); + receivedKey = client.put(jobId, is); + assertEquals(origKey, receivedKey); is.close(); is = null; @@ -287,6 +310,8 @@ public void testContentAddressableStream() { // Retrieve the data is = client.get(receivedKey); validateGet(is, testFile); + is = client.get(jobId, receivedKey); + validateGet(is, testFile); } catch (Exception e) { e.printStackTrace(); @@ -324,6 +349,13 @@ static void uploadJarFile(BlobServer blobServer, Configuration blobClientConfig) InetSocketAddress serverAddress = new InetSocketAddress("localhost", blobServer.getPort()); + uploadJarFile(serverAddress, blobClientConfig, testFile); + uploadJarFile(serverAddress, blobClientConfig, testFile); + } + + private static void uploadJarFile( + final InetSocketAddress serverAddress, final Configuration blobClientConfig, + final File testFile) throws IOException { List blobKeys = BlobClient.uploadJarFiles(serverAddress, blobClientConfig, Collections.singletonList(new Path(testFile.toURI()))); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java index 3c7711d86bc68..81304f45ad4b9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobRecoveryITCase.java @@ -94,11 +94,17 @@ public static void testBlobServerRecovery(final Configuration config, final Blob BlobKey[] keys = new BlobKey[2]; - // Put data - keys[0] = client.put(expected); // Request 1 - keys[1] = client.put(expected, 32, 256); // Request 2 + // Put job-unrelated data + keys[0] = client.put(null, expected); // Request 1 + keys[1] = client.put(null, expected, 32, 256); // Request 2 + // Put job-related data, verify that the checksums match JobID[] jobId = new JobID[] { new JobID(), new JobID() }; + BlobKey key; + key = client.put(jobId[0], expected); // Request 3 + assertEquals(keys[0], key); + key = client.put(jobId[1], expected, 32, 256); // Request 4 + assertEquals(keys[1], key); // check that the storage directory exists final Path blobServerPath = new Path(storagePath, "blob"); @@ -130,9 +136,31 @@ public static void testBlobServerRecovery(final Configuration config, final Blob } } + // Verify request 3 + try (InputStream is = client.get(jobId[0], keys[0])) { + byte[] actual = new byte[expected.length]; + BlobUtils.readFully(is, actual, 0, expected.length, null); + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], actual[i]); + } + } + + // Verify request 4 + try (InputStream is = client.get(jobId[1], keys[1])) { + byte[] actual = new byte[256]; + BlobUtils.readFully(is, actual, 0, 256, null); + + for (int i = 32, j = 0; i < 256; i++, j++) { + assertEquals(expected[i], actual[j]); + } + } + // Remove again client.delete(keys[0]); client.delete(keys[1]); + client.delete(jobId[0], keys[0]); + client.delete(jobId[1], keys[1]); // Verify everything is clean assertTrue("HA storage directory does not exist", fs.exists(new Path(storagePath))); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java index ce4574b7b29ce..d91aae420f852 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.concurrent.FlinkFutureException; @@ -39,6 +40,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; @@ -75,35 +77,45 @@ public void testDeleteSingleByBlobKey() { byte[] data = new byte[2000000]; rnd.nextBytes(data); - // put content addressable (like libraries) - BlobKey key = client.put(data); - assertNotNull(key); + // put job-unrelated (like libraries) + BlobKey key1 = client.put(null, data); + assertNotNull(key1); - // second item + // second job-unrelated item data[0] ^= 1; - BlobKey key2 = client.put(data); + BlobKey key2 = client.put(null, data); assertNotNull(key2); - assertNotEquals(key, key2); + assertNotEquals(key1, key2); + + // put job-related with same key1 as non-job-related + data[0] ^= 1; // back to the original data + final JobID jobId = new JobID(); + BlobKey key1b = client.put(jobId, data); + assertNotNull(key1b); + assertEquals(key1, key1b); // issue a DELETE request via the client - client.delete(key); + client.delete(key1); client.close(); client = new BlobClient(serverAddress, config); try { - client.get(key); + client.get(key1); fail("BLOB should have been deleted"); } catch (IOException e) { // expected } + ensureClientIsClosed(client); + + client = new BlobClient(serverAddress, config); try { - client.put(new byte[1]); - fail("client should be closed after erroneous operation"); + client.get(jobId, key1); } - catch (IllegalStateException e) { + catch (IOException e) { // expected + fail("Deleting a job-unrelated BLOB should not affect a job-related BLOB with the same key"); } // delete a file directly on the server @@ -125,8 +137,29 @@ public void testDeleteSingleByBlobKey() { } } + private static void ensureClientIsClosed(final BlobClient client) throws IOException { + try { + client.put(null, new byte[1]); + fail("client should be closed after erroneous operation"); + } + catch (IllegalStateException e) { + // expected + } finally { + client.close(); + } + } + + @Test + public void testDeleteAlreadyDeletedNoJob() { + testDeleteAlreadyDeleted(null); + } + @Test - public void testDeleteAlreadyDeletedByBlobKey() { + public void testDeleteAlreadyDeletedForJob() { + testDeleteAlreadyDeleted(new JobID()); + } + + private void testDeleteAlreadyDeleted(final JobID jobId) { BlobServer server = null; BlobClient client = null; BlobStore blobStore = new VoidBlobStore(); @@ -143,23 +176,27 @@ public void testDeleteAlreadyDeletedByBlobKey() { byte[] data = new byte[2000000]; rnd.nextBytes(data); - // put content addressable (like libraries) - BlobKey key = client.put(data); + // put file + BlobKey key = client.put(jobId, data); assertNotNull(key); - File blobFile = server.getStorageLocation(key); + File blobFile = server.getStorageLocation(jobId, key); assertTrue(blobFile.delete()); // issue a DELETE request via the client try { - client.delete(key); + deleteHelper(client, jobId, key); } catch (IOException e) { fail("DELETE operation should not fail if file is already deleted"); } // issue a DELETE request on the server - server.delete(key); + if (jobId == null) { + server.delete(key); + } else { + server.delete(jobId, key); + } } catch (Exception e) { e.printStackTrace(); @@ -170,8 +207,25 @@ public void testDeleteAlreadyDeletedByBlobKey() { } } + private static void deleteHelper(BlobClient client, JobID jobId, BlobKey key) throws IOException { + if (jobId == null) { + client.delete(key); + } else { + client.delete(jobId, key); + } + } + @Test - public void testDeleteByBlobKeyFails() { + public void testDeleteFailsNoJob() { + testDeleteFails(null); + } + + @Test + public void testDeleteFailsForJob() { + testDeleteFails(new JobID()); + } + + private void testDeleteFails(final JobID jobId) { assumeTrue(!OperatingSystem.isWindows()); //setWritable doesn't work on Windows. BlobServer server = null; @@ -193,29 +247,39 @@ public void testDeleteByBlobKeyFails() { rnd.nextBytes(data); // put content addressable (like libraries) - BlobKey key = client.put(data); + BlobKey key = client.put(jobId, data); assertNotNull(key); - blobFile = server.getStorageLocation(key); + blobFile = server.getStorageLocation(jobId, key); directory = blobFile.getParentFile(); assertTrue(blobFile.setWritable(false, false)); assertTrue(directory.setWritable(false, false)); // issue a DELETE request via the client - client.delete(key); + deleteHelper(client, jobId, key); // issue a DELETE request on the server - server.delete(key); + if (jobId == null) { + server.delete(key); + } else { + server.delete(jobId, key); + } // the file should still be there - server.getFile(key); + if (jobId == null) { + server.getFile(key); + } else { + server.getFile(jobId, key); + } } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } finally { if (blobFile != null && directory != null) { + //noinspection ResultOfMethodCallIgnored blobFile.setWritable(true, false); + //noinspection ResultOfMethodCallIgnored directory.setWritable(true, false); } cleanup(server, client); @@ -233,10 +297,29 @@ public void testDeleteByBlobKeyFails() { * broken. */ @Test - public void testConcurrentDeleteOperations() throws IOException, ExecutionException, InterruptedException { + public void testConcurrentDeleteOperationsNoJob() throws IOException, ExecutionException, InterruptedException { + testConcurrentDeleteOperations(null); + } + + /** + * FLINK-6020 + * + * Tests that concurrent delete operations don't interfere with each other. + * + * Note: The test checks that there cannot be two threads which have checked whether a given blob file exist + * and then one of them fails deleting it. Without the introduced lock, this situation should rarely happen + * and make this test fail. Thus, if this test should become "unstable", then the delete atomicity is most likely + * broken. + */ + @Test + public void testConcurrentDeleteOperationsForJob() throws IOException, ExecutionException, InterruptedException { + testConcurrentDeleteOperations(new JobID()); + } + + private void testConcurrentDeleteOperations(final JobID jobId) + throws IOException, InterruptedException, ExecutionException { final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); - final BlobStore blobStore = mock(BlobStore.class); final int concurrentDeleteOperations = 3; @@ -251,16 +334,16 @@ public void testConcurrentDeleteOperations() throws IOException, ExecutionExcept final BlobKey blobKey; try (BlobClient client = blobServer.createClient()) { - blobKey = client.put(data); + blobKey = client.put(jobId, data); } - assertTrue(blobServer.getStorageLocation(blobKey).exists()); + assertTrue(blobServer.getStorageLocation(jobId, blobKey).exists()); for (int i = 0; i < concurrentDeleteOperations; i++) { CompletableFuture deleteFuture = CompletableFuture.supplyAsync( () -> { try (BlobClient blobClient = blobServer.createClient()) { - blobClient.delete(blobKey); + deleteHelper(blobClient, jobId, blobKey); } catch (IOException e) { throw new FlinkFutureException("Could not delete the given blob key " + blobKey + '.', e); } @@ -278,13 +361,13 @@ public void testConcurrentDeleteOperations() throws IOException, ExecutionExcept // in case of no lock, one of the delete operations should eventually fail waitFuture.get(); - assertFalse(blobServer.getStorageLocation(blobKey).exists()); + assertFalse(blobServer.getStorageLocation(jobId, blobKey).exists()); } finally { executor.shutdownNow(); } } - private void cleanup(BlobServer server, BlobClient client) { + private static void cleanup(BlobServer server, BlobClient client) { if (client != null) { try { client.close(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java index bd27d702a9d18..5ad8d95928a1c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java @@ -20,6 +20,7 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.concurrent.FlinkFutureException; @@ -69,7 +70,28 @@ public class BlobServerGetTest extends TestLogger { public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Test - public void testGetFailsDuringLookup() throws IOException { + public void testGetFailsDuringLookup1() throws IOException { + testGetFailsDuringLookup(null, new JobID()); + } + + @Test + public void testGetFailsDuringLookup2() throws IOException { + testGetFailsDuringLookup(new JobID(), new JobID()); + } + + @Test + public void testGetFailsDuringLookup3() throws IOException { + testGetFailsDuringLookup(new JobID(), null); + } + + /** + * Checks the correct result if a GET operation fails during the lookup of the file. + * + * @param jobId1 first job ID or null if job-unrelated + * @param jobId2 second job ID different to jobId1 + */ + private void testGetFailsDuringLookup(final JobID jobId1, final JobID jobId2) + throws IOException { BlobServer server = null; BlobClient client = null; @@ -86,20 +108,29 @@ public void testGetFailsDuringLookup() throws IOException { rnd.nextBytes(data); // put content addressable (like libraries) - BlobKey key = client.put(data); + BlobKey key = client.put(jobId1, data); assertNotNull(key); - // delete all files to make sure that GET requests fail - File blobFile = server.getStorageLocation(key); + // delete file to make sure that GET requests fail + File blobFile = server.getStorageLocation(jobId1, key); assertTrue(blobFile.delete()); // issue a GET request that fails - try { - client.get(key); - fail("This should not succeed."); - } catch (IOException e) { - // expected - } + client = verifyDeleted(client, jobId1, key, serverAddress, config); + + BlobKey key2 = client.put(jobId2, data); + assertNotNull(key); + assertEquals(key, key2); + // request for jobId2 should succeed + getFileHelper(client, jobId2, key); + // request for jobId1 should still fail + client = verifyDeleted(client, jobId1, key, serverAddress, config); + + // same checks as for jobId1 but for jobId2 should also work: + blobFile = server.getStorageLocation(jobId2, key); + assertTrue(blobFile.delete()); + client = verifyDeleted(client, jobId2, key, serverAddress, config); + } finally { if (client != null) { client.close(); @@ -110,8 +141,51 @@ public void testGetFailsDuringLookup() throws IOException { } } + /** + * Checks that the given blob does not exist anymore. + * + * @param client + * BLOB client to use for connecting to the BLOB server + * @param jobId + * job ID or null if job-unrelated + * @param key + * key identifying the BLOB to request + * @param serverAddress + * BLOB server address + * @param config + * client config + * + * @return a new client (since the old one is being closed on failure) + */ + private static BlobClient verifyDeleted( + BlobClient client, JobID jobId, BlobKey key, + InetSocketAddress serverAddress, Configuration config) throws IOException { + try { + getFileHelper(client, jobId, key); + fail("This should not succeed."); + } catch (IOException e) { + // expected + } + // need a new client (old ony closed due to failure + return new BlobClient(serverAddress, config); + } + + @Test + public void testGetFailsDuringStreamingNoJob() throws IOException { + testGetFailsDuringStreaming(null); + } + @Test - public void testGetFailsDuringStreaming() throws IOException { + public void testGetFailsDuringStreamingForJob() throws IOException { + testGetFailsDuringStreaming(new JobID()); + } + + /** + * Checks the correct result if a GET operation fails during the file download. + * + * @param jobId job ID or null if job-unrelated + */ + private void testGetFailsDuringStreaming(final JobID jobId) throws IOException { BlobServer server = null; BlobClient client = null; @@ -128,11 +202,11 @@ public void testGetFailsDuringStreaming() throws IOException { rnd.nextBytes(data); // put content addressable (like libraries) - BlobKey key = client.put(data); + BlobKey key = client.put(jobId, data); assertNotNull(key); // issue a GET request that succeeds - InputStream is = client.get(key); + InputStream is = getFileHelper(client, jobId, key); byte[] receiveBuffer = new byte[data.length]; int firstChunkLen = 50000; @@ -169,8 +243,22 @@ public void testGetFailsDuringStreaming() throws IOException { * Tests that concurrent get operations don't concurrently access the BlobStore to download a blob. */ @Test - public void testConcurrentGetOperations() throws IOException, ExecutionException, InterruptedException { + public void testConcurrentGetOperationsNoJob() throws IOException, ExecutionException, InterruptedException { + testConcurrentGetOperations(null); + } + /** + * FLINK-6020 + * + * Tests that concurrent get operations don't concurrently access the BlobStore to download a blob. + */ + @Test + public void testConcurrentGetOperationsForJob() throws IOException, ExecutionException, InterruptedException { + testConcurrentGetOperations(new JobID()); + } + + private void testConcurrentGetOperations(final JobID jobId) + throws IOException, InterruptedException, ExecutionException { final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); @@ -191,14 +279,14 @@ public void testConcurrentGetOperations() throws IOException, ExecutionException new Answer() { @Override public Object answer(InvocationOnMock invocation) throws Throwable { - File targetFile = (File) invocation.getArguments()[1]; + File targetFile = (File) invocation.getArguments()[2]; FileUtils.copyInputStreamToFile(bais, targetFile); return null; } } - ).when(blobStore).get(any(BlobKey.class), any(File.class)); + ).when(blobStore).get(any(JobID.class), any(BlobKey.class), any(File.class)); final ExecutorService executor = Executors.newFixedThreadPool(numberConcurrentGetOperations); @@ -207,7 +295,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { CompletableFuture getOperation = CompletableFuture.supplyAsync( () -> { try (BlobClient blobClient = blobServer.createClient(); - InputStream inputStream = blobClient.get(blobKey)) { + InputStream inputStream = getFileHelper(blobClient, jobId, blobKey)) { byte[] buffer = new byte[data.length]; IOUtils.readFully(inputStream, buffer); @@ -241,9 +329,18 @@ public Object answer(InvocationOnMock invocation) throws Throwable { } // verify that we downloaded the requested blob exactly once from the BlobStore - verify(blobStore, times(1)).get(eq(blobKey), any(File.class)); + verify(blobStore, times(1)).get(eq(jobId), eq(blobKey), any(File.class)); } finally { executor.shutdownNow(); } } + + static InputStream getFileHelper(BlobClient blobClient, JobID jobId, BlobKey blobKey) + throws IOException { + if (jobId == null) { + return blobClient.get(blobKey); + } else { + return blobClient.get(jobId, blobKey); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java index c4791672a73e7..f55adb7a34c3c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CheckedThread; @@ -45,6 +46,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import static org.apache.flink.runtime.blob.BlobServerGetTest.getFileHelper; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -71,28 +73,43 @@ public class BlobServerPutTest extends TestLogger { // --- concurrency tests for utility methods which could fail during the put operation --- /** - * Checked thread that calls {@link BlobServer#getStorageLocation(BlobKey)} + * Checked thread that calls {@link BlobServer#getStorageLocation(JobID, BlobKey)}. */ public static class ContentAddressableGetStorageLocation extends CheckedThread { private final BlobServer server; + private final JobID jobId; private final BlobKey key; - public ContentAddressableGetStorageLocation(BlobServer server, BlobKey key) { + public ContentAddressableGetStorageLocation(BlobServer server, JobID jobId, BlobKey key) { this.server = server; + this.jobId = jobId; this.key = key; } @Override public void go() throws Exception { - server.getStorageLocation(key); + server.getStorageLocation(jobId, key); } } /** - * Tests concurrent calls to {@link BlobServer#getStorageLocation(BlobKey)}. + * Tests concurrent calls to {@link BlobServer#getStorageLocation(JobID, BlobKey)}. */ @Test - public void testServerContentAddressableGetStorageLocationConcurrent() throws Exception { + public void testServerContentAddressableGetStorageLocationConcurrentNoJob() throws Exception { + testServerContentAddressableGetStorageLocationConcurrent(null); + } + + /** + * Tests concurrent calls to {@link BlobServer#getStorageLocation(JobID, BlobKey)}. + */ + @Test + public void testServerContentAddressableGetStorageLocationConcurrentForJob() throws Exception { + testServerContentAddressableGetStorageLocationConcurrent(new JobID()); + } + + private void testServerContentAddressableGetStorageLocationConcurrent(final JobID jobId) + throws Exception { final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); @@ -101,9 +118,9 @@ public void testServerContentAddressableGetStorageLocationConcurrent() throws Ex try { BlobKey key = new BlobKey(); CheckedThread[] threads = new CheckedThread[] { - new ContentAddressableGetStorageLocation(server, key), - new ContentAddressableGetStorageLocation(server, key), - new ContentAddressableGetStorageLocation(server, key) + new ContentAddressableGetStorageLocation(server, jobId, key), + new ContentAddressableGetStorageLocation(server, jobId, key), + new ContentAddressableGetStorageLocation(server, jobId, key) }; checkedThreadSimpleTest(threads); } finally { @@ -134,7 +151,27 @@ protected void checkedThreadSimpleTest(CheckedThread[] threads) // -------------------------------------------------------------------------------------------- @Test - public void testPutBufferSuccessful() throws IOException { + public void testPutBufferSuccessfulGet1() throws IOException { + testPutBufferSuccessfulGet(null, null); + } + + @Test + public void testPutBufferSuccessfulGet2() throws IOException { + testPutBufferSuccessfulGet(null, new JobID()); + } + + @Test + public void testPutBufferSuccessfulGet3() throws IOException { + testPutBufferSuccessfulGet(new JobID(), new JobID()); + } + + @Test + public void testPutBufferSuccessfulGet4() throws IOException { + testPutBufferSuccessfulGet(new JobID(), null); + } + + private void testPutBufferSuccessfulGet(final JobID jobId1, final JobID jobId2) + throws IOException { BlobServer server = null; BlobClient client = null; @@ -150,17 +187,66 @@ public void testPutBufferSuccessful() throws IOException { byte[] data = new byte[2000000]; rnd.nextBytes(data); - // put content addressable (like libraries) - BlobKey key1 = client.put(data); - assertNotNull(key1); + // put data for jobId1 and verify + BlobKey key1a = client.put(jobId1, data); + assertNotNull(key1a); + + BlobKey key1b = client.put(jobId1, data, 10, 44); + assertNotNull(key1b); + + testPutBufferSuccessfulGet(jobId1, key1a, key1b, data, serverAddress, config); + + // now put data for jobId2 and verify that both are ok + BlobKey key2a = client.put(jobId2, data); + assertNotNull(key2a); + assertEquals(key1a, key2a); + + BlobKey key2b = client.put(jobId2, data, 10, 44); + assertNotNull(key2b); + assertEquals(key1b, key2b); + + + testPutBufferSuccessfulGet(jobId1, key1a, key1b, data, serverAddress, config); + testPutBufferSuccessfulGet(jobId2, key2a, key2b, data, serverAddress, config); + + + } finally { + if (client != null) { + client.close(); + } + if (server != null) { + server.close(); + } + } + } - BlobKey key2 = client.put(data, 10, 44); - assertNotNull(key2); + /** + * GET the data stored at the two keys and check that it is equal to data. + * + * @param jobId + * job ID or null if job-unrelated + * @param key1 + * first key + * @param key2 + * second key + * @param data + * expected data + * @param serverAddress + * BlobServer address to connect to + * @param config + * client configuration + */ + private static void testPutBufferSuccessfulGet( + JobID jobId, BlobKey key1, BlobKey key2, byte[] data, + InetSocketAddress serverAddress, Configuration config) throws IOException { - // --- GET the data and check that it is equal --- + BlobClient client = new BlobClient(serverAddress, config); + InputStream is1 = null; + InputStream is2 = null; + try { // one get request on the same client - InputStream is1 = client.get(key2); + is1 = getFileHelper(client, jobId, key2); byte[] result1 = new byte[44]; BlobUtils.readFully(is1, result1, 0, result1.length, null); is1.close(); @@ -173,24 +259,31 @@ public void testPutBufferSuccessful() throws IOException { client.close(); client = new BlobClient(serverAddress, config); - InputStream is2 = client.get(key1); - byte[] result2 = new byte[data.length]; - BlobUtils.readFully(is2, result2, 0, result2.length, null); + is2 = getFileHelper(client, jobId, key1); + BlobClientTest.validateGet(is2, data); is2.close(); - assertArrayEquals(data, result2); } finally { - if (client != null) { - client.close(); + if (is1 != null) { + is1.close(); } - if (server != null) { - server.close(); + if (is2 != null) { + is1.close(); } + client.close(); } } + @Test + public void testPutStreamSuccessfulNoJob() throws IOException { + testPutStreamSuccessful(null); + } @Test - public void testPutStreamSuccessful() throws IOException { + public void testPutStreamSuccessfulForJob() throws IOException { + testPutStreamSuccessful(new JobID()); + } + + private void testPutStreamSuccessful(final JobID jobId) throws IOException { BlobServer server = null; BlobClient client = null; @@ -208,7 +301,12 @@ public void testPutStreamSuccessful() throws IOException { // put content addressable (like libraries) { - BlobKey key1 = client.put(new ByteArrayInputStream(data)); + BlobKey key1; + if (jobId == null) { + key1 = client.put(new ByteArrayInputStream(data)); + } else { + key1 = client.put(jobId, new ByteArrayInputStream(data)); + } assertNotNull(key1); } } finally { @@ -226,7 +324,16 @@ public void testPutStreamSuccessful() throws IOException { } @Test - public void testPutChunkedStreamSuccessful() throws IOException { + public void testPutChunkedStreamSuccessfulNoJob() throws IOException { + testPutChunkedStreamSuccessful(null); + } + + @Test + public void testPutChunkedStreamSuccessfulForJob() throws IOException { + testPutChunkedStreamSuccessful(new JobID()); + } + + private void testPutChunkedStreamSuccessful(final JobID jobId) throws IOException { BlobServer server = null; BlobClient client = null; @@ -244,7 +351,12 @@ public void testPutChunkedStreamSuccessful() throws IOException { // put content addressable (like libraries) { - BlobKey key1 = client.put(new ChunkedInputStream(data, 19)); + BlobKey key1; + if (jobId == null) { + key1 = client.put(new ChunkedInputStream(data, 19)); + } else { + key1 = client.put(jobId, new ChunkedInputStream(data, 19)); + } assertNotNull(key1); } } finally { @@ -258,7 +370,16 @@ public void testPutChunkedStreamSuccessful() throws IOException { } @Test - public void testPutBufferFails() throws IOException { + public void testPutBufferFailsNoJob() throws IOException { + testPutBufferFails(null); + } + + @Test + public void testPutBufferFailsForJob() throws IOException { + testPutBufferFails(new JobID()); + } + + private void testPutBufferFails(final JobID jobId) throws IOException { assumeTrue(!OperatingSystem.isWindows()); //setWritable doesn't work on Windows. BlobServer server = null; @@ -285,7 +406,7 @@ public void testPutBufferFails() throws IOException { // put content addressable (like libraries) try { - client.put(data); + client.put(jobId, data); fail("This should fail."); } catch (IOException e) { @@ -293,7 +414,7 @@ public void testPutBufferFails() throws IOException { } try { - client.put(data); + client.put(jobId, data); fail("Client should be closed"); } catch (IllegalStateException e) { @@ -320,7 +441,22 @@ public void testPutBufferFails() throws IOException { * Tests that concurrent put operations will only upload the file once to the {@link BlobStore}. */ @Test - public void testConcurrentPutOperations() throws IOException, ExecutionException, InterruptedException { + public void testConcurrentPutOperationsNoJob() throws IOException, ExecutionException, InterruptedException { + testConcurrentPutOperations(null); + } + + /** + * FLINK-6020 + * + * Tests that concurrent put operations will only upload the file once to the {@link BlobStore}. + */ + @Test + public void testConcurrentPutOperationsForJob() throws IOException, ExecutionException, InterruptedException { + testConcurrentPutOperations(new JobID()); + } + + private void testConcurrentPutOperations(final JobID jobId) + throws IOException, InterruptedException, ExecutionException { final Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); @@ -331,7 +467,7 @@ public void testConcurrentPutOperations() throws IOException, ExecutionException final CountDownLatch countDownLatch = new CountDownLatch(concurrentPutOperations); final byte[] data = new byte[dataSize]; - ArrayList> allFutures = new ArrayList(concurrentPutOperations); + ArrayList> allFutures = new ArrayList<>(concurrentPutOperations); ExecutorService executor = Executors.newFixedThreadPool(concurrentPutOperations); @@ -342,7 +478,13 @@ public void testConcurrentPutOperations() throws IOException, ExecutionException CompletableFuture putFuture = CompletableFuture.supplyAsync( () -> { try (BlobClient blobClient = blobServer.createClient()) { - return blobClient.put(new BlockingInputStream(countDownLatch, data)); + if (jobId == null) { + return blobClient + .put(new BlockingInputStream(countDownLatch, data)); + } else { + return blobClient + .put(jobId, new BlockingInputStream(countDownLatch, data)); + } } catch (IOException e) { throw new FlinkFutureException("Could not upload blob.", e); } @@ -369,7 +511,7 @@ public void testConcurrentPutOperations() throws IOException, ExecutionException } // check that we only uploaded the file once to the blob store - verify(blobStore, times(1)).put(any(File.class), eq(blobKey)); + verify(blobStore, times(1)).put(any(File.class), eq(jobId), eq(blobKey)); } finally { executor.shutdownNow(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java index 2987c3976bea5..e449aab822854 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java @@ -18,11 +18,9 @@ package org.apache.flink.runtime.blob; -import static org.junit.Assert.assertTrue; -import static org.junit.Assume.assumeTrue; -import static org.mockito.Mockito.mock; - +import org.apache.flink.api.common.JobID; import org.apache.flink.util.OperatingSystem; + import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -32,6 +30,10 @@ import java.io.File; import java.io.IOException; +import static org.junit.Assert.assertTrue; +import static org.junit.Assume.assumeTrue; +import static org.mockito.Mockito.mock; + public class BlobUtilsTest { private final static String CANNOT_CREATE_THIS = "cannot-create-this"; @@ -62,12 +64,18 @@ public void after() { public void testExceptionOnCreateStorageDirectoryFailure() throws IOException { // Should throw an Exception - BlobUtils.initStorageDirectory(new File(blobUtilsTestDirectory, CANNOT_CREATE_THIS).getAbsolutePath()); + BlobUtils.initLocalStorageDirectory(new File(blobUtilsTestDirectory, CANNOT_CREATE_THIS).getAbsolutePath()); + } + + @Test(expected = Exception.class) + public void testExceptionOnCreateCacheDirectoryFailureNoJob() { + // Should throw an Exception + BlobUtils.getStorageLocation(new File(blobUtilsTestDirectory, CANNOT_CREATE_THIS), null, mock(BlobKey.class)); } @Test(expected = Exception.class) - public void testExceptionOnCreateCacheDirectoryFailure() { + public void testExceptionOnCreateCacheDirectoryFailureForJob() { // Should throw an Exception - BlobUtils.getStorageLocation(new File(blobUtilsTestDirectory, CANNOT_CREATE_THIS), mock(BlobKey.class)); + BlobUtils.getStorageLocation(new File(blobUtilsTestDirectory, CANNOT_CREATE_THIS), new JobID(), mock(BlobKey.class)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java index 476fdcbf3c458..b43a307447d83 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java @@ -73,17 +73,20 @@ public void testLibraryCacheManagerJobCleanup() throws IOException, InterruptedE InetSocketAddress blobSocketAddress = new InetSocketAddress(server.getPort()); BlobClient bc = new BlobClient(blobSocketAddress, config); - keys.add(bc.put(buf)); + // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager + JobID jobId = null; + + keys.add(bc.put(jobId, buf)); buf[0] += 1; - keys.add(bc.put(buf)); + keys.add(bc.put(jobId, buf)); bc.close(); - long cleanupInterval = 1000l; + long cleanupInterval = 1000L; libraryCacheManager = new BlobLibraryCacheManager(server, cleanupInterval); libraryCacheManager.registerJob(jid, keys, Collections.emptyList()); - assertEquals(2, checkFilesExist(keys, server, true)); + assertEquals(2, checkFilesExist(jobId, keys, server, true)); assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries()); assertEquals(1, libraryCacheManager.getNumberOfReferenceHolders(jid)); @@ -105,17 +108,25 @@ public void testLibraryCacheManagerJobCleanup() throws IOException, InterruptedE assertEquals(0, libraryCacheManager.getNumberOfReferenceHolders(jid)); // the blob cache should no longer contain the files - assertEquals(0, checkFilesExist(keys, server, false)); + assertEquals(0, checkFilesExist(jobId, keys, server, false)); try { - server.getFile(keys.get(0)); - fail("name-addressable BLOB should have been deleted"); + if (jobId == null) { + server.getFile(keys.get(0)); + } else { + server.getFile(jobId, keys.get(0)); + } + fail("BLOB should have been deleted"); } catch (IOException e) { // expected } try { - server.getFile(keys.get(1)); - fail("name-addressable BLOB should have been deleted"); + if (jobId == null) { + server.getFile(keys.get(1)); + } else { + server.getFile(jobId, keys.get(1)); + } + fail("BLOB should have been deleted"); } catch (IOException e) { // expected } @@ -150,16 +161,20 @@ public void testLibraryCacheManagerJobCleanup() throws IOException, InterruptedE * @param doThrow * whether exceptions should be ignored (false), or throws (true) * - * @return number of files we were able to retrieve via {@link BlobService#getFile(BlobKey)} + * @return number of files we were able to retrieve via {@link BlobService#getFile} */ private static int checkFilesExist( - List keys, BlobService blobService, boolean doThrow) + JobID jobId, List keys, BlobService blobService, boolean doThrow) throws IOException { int numFiles = 0; for (BlobKey key : keys) { try { - blobService.getFile(key); + if (jobId == null) { + blobService.getFile(key); + } else { + blobService.getFile(jobId, key); + } ++numFiles; } catch (IOException e) { if (doThrow) { @@ -196,22 +211,26 @@ public void testLibraryCacheManagerTaskCleanup() throws IOException, Interrupted InetSocketAddress blobSocketAddress = new InetSocketAddress(server.getPort()); BlobClient bc = new BlobClient(blobSocketAddress, config); - keys.add(bc.put(buf)); + // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager +// JobID jobId = new JobID(); + JobID jobId = null; + + keys.add(bc.put(jobId, buf)); buf[0] += 1; - keys.add(bc.put(buf)); + keys.add(bc.put(jobId, buf)); - long cleanupInterval = 1000l; + long cleanupInterval = 1000L; libraryCacheManager = new BlobLibraryCacheManager(server, cleanupInterval); libraryCacheManager.registerTask(jid, executionId1, keys, Collections.emptyList()); libraryCacheManager.registerTask(jid, executionId2, keys, Collections.emptyList()); - assertEquals(2, checkFilesExist(keys, server, true)); + assertEquals(2, checkFilesExist(jobId, keys, server, true)); assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries()); assertEquals(2, libraryCacheManager.getNumberOfReferenceHolders(jid)); libraryCacheManager.unregisterTask(jid, executionId1); - assertEquals(2, checkFilesExist(keys, server, true)); + assertEquals(2, checkFilesExist(jobId, keys, server, true)); assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries()); assertEquals(1, libraryCacheManager.getNumberOfReferenceHolders(jid)); @@ -233,7 +252,7 @@ public void testLibraryCacheManagerTaskCleanup() throws IOException, Interrupted assertEquals(0, libraryCacheManager.getNumberOfReferenceHolders(jid)); // the blob cache should no longer contain the files - assertEquals(0, checkFilesExist(keys, server, false)); + assertEquals(0, checkFilesExist(jobId, keys, server, false)); bc.close(); } finally { @@ -269,10 +288,13 @@ public void testRegisterAndDownload() throws IOException { InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort()); cache = new BlobCache(serverAddress, config, new VoidBlobStore()); + // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager + JobID jobId = null; + // upload some meaningless data to the server BlobClient uploader = new BlobClient(serverAddress, config); - BlobKey dataKey1 = uploader.put(new byte[]{1, 2, 3, 4, 5, 6, 7, 8}); - BlobKey dataKey2 = uploader.put(new byte[]{11, 12, 13, 14, 15, 16, 17, 18}); + BlobKey dataKey1 = uploader.put(jobId, new byte[]{1, 2, 3, 4, 5, 6, 7, 8}); + BlobKey dataKey2 = uploader.put(jobId, new byte[]{11, 12, 13, 14, 15, 16, 17, 18}); uploader.close(); BlobLibraryCacheManager libCache = new BlobLibraryCacheManager(cache, 1000000000L); @@ -316,11 +338,12 @@ public void testRegisterAndDownload() throws IOException { fail("Should fail with an IllegalStateException"); } catch (IllegalStateException e) { - // that#s what we want + // that's what we want } } - cacheDir = new File(cache.getStorageDir(), "cache"); + // see BlobUtils for the directory layout + cacheDir = new File(cache.getStorageDir(), "no_job"); assertTrue(cacheDir.exists()); // make sure no further blobs can be downloaded by removing the write diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java index b19835bc8c4a5..2f6738da33fd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java @@ -91,10 +91,14 @@ public void testRecoveryRegisterAndDownload() throws Exception { List keys = new ArrayList<>(2); + JobID jobId = new JobID(); + // TODO: replace+adapt by jobId after adapting the BlobLibraryCacheManager + JobID blobJobId = null; + // Upload some data (libraries) try (BlobClient client = new BlobClient(serverAddress[0], config)) { - keys.add(client.put(expected)); // Request 1 - keys.add(client.put(expected, 32, 256)); // Request 2 + keys.add(client.put(blobJobId, expected)); // Request 1 + keys.add(client.put(blobJobId, expected, 32, 256)); // Request 2 } // The cache @@ -102,7 +106,6 @@ public void testRecoveryRegisterAndDownload() throws Exception { libCache = new BlobLibraryCacheManager(cache, 3600 * 1000); // Register uploaded libraries - JobID jobId = new JobID(); ExecutionAttemptID executionId = new ExecutionAttemptID(); libServer[0].registerTask(jobId, executionId, keys, Collections.emptyList()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java index 79b9c1c56765d..3c75971f5c7c8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.jobmanager; import akka.actor.ActorSystem; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.akka.AkkaUtils; @@ -136,9 +137,11 @@ public void testFailureWhenJarBlobsMissing() { // upload two dummy bytes and add their keys to the job graph as dependencies BlobKey key1, key2; BlobClient bc = new BlobClient(new InetSocketAddress("localhost", blobPort), jmConfig); + // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager + JobID jobId = null; try { - key1 = bc.put(new byte[10]); - key2 = bc.put(new byte[10]); + key1 = bc.put(jobId, new byte[10]); + key2 = bc.put(jobId, new byte[10]); // delete one of the blobs to make sure that the startup failed bc.delete(key2); From 4c0fa6edf52ac2ec2a902a44022d99397755a4f1 Mon Sep 17 00:00:00 2001 From: Svend Vanderveken Date: Sat, 12 Aug 2017 11:11:36 +0200 Subject: [PATCH 007/129] [FLINK-7434][doc] scafolding with "sbt new" Since the generated project is an sbt project, it is much more straightfoward for the user to create it with the new "sbt new" command than by using giter8 (which requires to install giter8 just for that purpose). This closes #4531. --- docs/quickstart/scala_api_quickstart.md | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/docs/quickstart/scala_api_quickstart.md b/docs/quickstart/scala_api_quickstart.md index abf6021a8f901..9e563ed25eb2c 100644 --- a/docs/quickstart/scala_api_quickstart.md +++ b/docs/quickstart/scala_api_quickstart.md @@ -41,25 +41,20 @@ These templates help you to set up the project structure and to create the initi ### Create Project +You can scafold a new project via either of the following two methods: +

-
- {% highlight bash %} - $ g8 tillrohrmann/flink-project - {% endhighlight %} - This will create a Flink project in the specified project directory from the flink-project template. - If you haven't installed giter8, then please follow this installation guide. -
-
+
{% highlight bash %} - $ git clone https://github.com/tillrohrmann/flink-project.git + $ sbt new tillrohrmann/flink-project.g8 {% endhighlight %} - This will create the Flink project in the directory flink-project. + This will will prompt you for a couple of parameters (project name, flink version...) and then create a Flink project from the flink-project template. + You need sbt >= 0.13.13 to execute this command. You can follow this installation guide to obtain it if necessary.
{% highlight bash %} From 3c84f8b9b9fa36d6fbb2d340015ed151a9771e1f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 10 Aug 2017 14:07:09 +0200 Subject: [PATCH 008/129] [FLINK-6180] [rpc] Remove TestingSerialRpcService The TestingSerialRpcService produces thread interleavings which are not happening when being executed with a proper RpcService implementation. Due to this the test cases can fail or succeed wrongly. In order to avoid this problem, this commit removes the TestingSerialRpcService and adapts all existing tests which used it before. Remove TestingSerialRpcService from MesosResourceManagerTest Remove TestingSerialRpcService from ResourceManagerJobMasterTest Remove TestingSerialRpcService from ResourceManagerTaskExecutorTest Remove TestingSerialRpcService from ResourceManagerTest Remove SerialTestingRpcService from JobMasterTest Remove TestingSerialRpcService from TaskExecutorITCase Remove TestingSerialRpcService from TaskExecutorTest Remove TestingSerialRpcService from SlotPoolTest Delete TestingSerialRpcService This closes #4516. --- .../MesosResourceManagerTest.java | 13 +- .../flink/runtime/instance/SlotPool.java | 4 +- .../runtime/instance/SlotPoolGateway.java | 3 +- .../resourcemanager/ResourceManager.java | 2 - .../flink/runtime/rpc/akka/AkkaRpcActor.java | 3 + .../taskexecutor/slot/TaskSlotTable.java | 2 - .../clusterframework/ResourceManagerTest.java | 31 +- .../flink/runtime/instance/SlotPoolTest.java | 334 +++++++------ .../runtime/jobmaster/JobMasterTest.java | 27 +- .../ResourceManagerHATest.java | 31 +- .../ResourceManagerJobMasterTest.java | 33 +- .../ResourceManagerTaskExecutorTest.java | 25 +- .../runtime/rpc/TestingSerialRpcService.java | 440 ------------------ .../taskexecutor/TaskExecutorITCase.java | 21 +- .../taskexecutor/TaskExecutorTest.java | 90 ++-- 15 files changed, 382 insertions(+), 677 deletions(-) delete mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java 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 f9e35a9e6b2f2..e81a2de3f1b47 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 @@ -58,7 +58,7 @@ import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; @@ -205,14 +205,14 @@ protected void closeTaskManagerConnection(ResourceID resourceID, Exception cause static class Context implements AutoCloseable { // services - TestingSerialRpcService rpcService; + TestingRpcService rpcService; TestingFatalErrorHandler fatalErrorHandler; MockMesosResourceManagerRuntimeServices rmServices; // RM ResourceManagerConfiguration rmConfiguration; ResourceID rmResourceID; - static final String RM_ADDRESS = "/resourceManager"; + static final String RM_ADDRESS = "resourceManager"; TestingMesosResourceManager resourceManager; // domain objects for test purposes @@ -239,7 +239,7 @@ static class Context implements AutoCloseable { * Create mock RM dependencies. */ Context() throws Exception { - rpcService = new TestingSerialRpcService(); + rpcService = new TestingRpcService(); fatalErrorHandler = new TestingFatalErrorHandler(); rmServices = new MockMesosResourceManagerRuntimeServices(); @@ -300,6 +300,7 @@ class MockResourceManagerRuntimeServices { public final TestingLeaderElectionService rmLeaderElectionService; public final JobLeaderIdService jobLeaderIdService; public final SlotManager slotManager; + public final CompletableFuture slotManagerStarted; public ResourceManagerActions rmActions; public UUID rmLeaderSessionId; @@ -312,6 +313,7 @@ class MockResourceManagerRuntimeServices { heartbeatServices = new TestingHeartbeatServices(5L, 5L, scheduledExecutor); metricRegistry = mock(MetricRegistry.class); slotManager = mock(SlotManager.class); + slotManagerStarted = new CompletableFuture<>(); jobLeaderIdService = new JobLeaderIdService( highAvailabilityServices, rpcService.getScheduledExecutor(), @@ -321,6 +323,7 @@ class MockResourceManagerRuntimeServices { @Override public Object answer(InvocationOnMock invocation) throws Throwable { rmActions = invocation.getArgumentAt(2, ResourceManagerActions.class); + slotManagerStarted.complete(true); return null; } }).when(slotManager).start(any(UUID.class), any(Executor.class), any(ResourceManagerActions.class)); @@ -426,6 +429,7 @@ public void registerJobMaster(MockJobMaster jobMaster) throws Exception { */ public MesosWorkerStore.Worker allocateWorker(Protos.TaskID taskID, ResourceProfile resourceProfile) throws Exception { when(rmServices.workerStore.newTaskID()).thenReturn(taskID); + rmServices.slotManagerStarted.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); rmServices.rmActions.allocateResource(resourceProfile); MesosWorkerStore.Worker expected = MesosWorkerStore.Worker.newWorker(taskID, resourceProfile); @@ -501,6 +505,7 @@ public void testRequestNewWorkers() throws Exception { // allocate a worker when(rmServices.workerStore.newTaskID()).thenReturn(task1).thenThrow(new AssertionFailedError()); + rmServices.slotManagerStarted.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); rmServices.rmActions.allocateResource(resourceProfile1); // verify that a new worker was persisted, the internal state was updated, the task router was notified, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java index de2b3e5f2aafb..bf3de25859c42 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java @@ -613,7 +613,7 @@ public void registerTaskManager(final ResourceID resourceID) { * @param resourceID The id of the TaskManager */ @Override - public void releaseTaskManager(final ResourceID resourceID) { + public CompletableFuture releaseTaskManager(final ResourceID resourceID) { if (registeredTaskManagers.remove(resourceID)) { availableSlots.removeAllForTaskManager(resourceID); @@ -622,6 +622,8 @@ public void releaseTaskManager(final ResourceID resourceID) { slot.releaseSlot(); } } + + return CompletableFuture.completedFuture(Acknowledge.get()); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java index 8d4f2a514f51c..32a9af5c9ef06 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; @@ -74,7 +75,7 @@ public interface SlotPoolGateway extends RpcGateway { void registerTaskManager(ResourceID resourceID); - void releaseTaskManager(ResourceID resourceID); + CompletableFuture releaseTaskManager(ResourceID resourceID); CompletableFuture offerSlot(AllocatedSlot slot); 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 c2b0590e49b81..e8ec0e006d21b 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 @@ -899,8 +899,6 @@ public void run() { clearState(); slotManager.suspend(); - - leaderSessionId = null; } }); } 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 f557447dee9db..d51607ecb5d97 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 @@ -189,6 +189,9 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { if (rpcMethod != null) { try { + // this supports declaration of anonymous classes + rpcMethod.setAccessible(true); + if (rpcMethod.getReturnType().equals(Void.TYPE)) { // No return value to send back rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java index 5c51c7c359b01..3634df00d88e9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java @@ -291,8 +291,6 @@ public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFo TaskSlot taskSlot = getTaskSlot(allocationId); if (taskSlot != null) { - LOG.info("Free slot {}.", allocationId, cause); - final JobID jobId = taskSlot.getJobId(); if (taskSlot.markFree()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java index 3ca0327c02199..9ad251b8a476e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java @@ -48,9 +48,10 @@ import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; @@ -64,6 +65,8 @@ import org.junit.BeforeClass; import org.junit.Test; import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + import scala.Option; import java.util.ArrayList; @@ -93,6 +96,8 @@ public class ResourceManagerTest extends TestLogger { private static Configuration config = new Configuration(); + private final Time timeout = Time.seconds(10L); + private TestingHighAvailabilityServices highAvailabilityServices; private TestingLeaderRetrievalService jobManagerLeaderRetrievalService; @@ -479,7 +484,7 @@ public void testHeartbeatTimeoutWithTaskExecutor() throws Exception { final ResourceID resourceManagerResourceID = ResourceID.generate(); final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); - final TestingSerialRpcService rpcService = new TestingSerialRpcService(); + final TestingRpcService rpcService = new TestingRpcService(); rpcService.registerGateway(taskManagerAddress, taskExecutorGateway); final ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration( @@ -519,18 +524,20 @@ public void testHeartbeatTimeoutWithTaskExecutor() throws Exception { resourceManager.start(); + final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); + final UUID rmLeaderSessionId = UUID.randomUUID(); rmLeaderElectionService.isLeader(rmLeaderSessionId); final SlotReport slotReport = new SlotReport(); // test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time - CompletableFuture successfulFuture = resourceManager.registerTaskExecutor( + CompletableFuture successfulFuture = rmGateway.registerTaskExecutor( rmLeaderSessionId, taskManagerAddress, taskManagerResourceID, slotReport, - Time.milliseconds(0L)); - RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS); + timeout); + RegistrationResponse response = successfulFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(response instanceof TaskExecutorRegistrationSuccess); ArgumentCaptor heartbeatRunnableCaptor = ArgumentCaptor.forClass(Runnable.class); @@ -557,7 +564,7 @@ public void testHeartbeatTimeoutWithTaskExecutor() throws Exception { // run the timeout runnable to simulate a heartbeat timeout timeoutRunnable.run(); - verify(taskExecutorGateway).disconnectResourceManager(any(TimeoutException.class)); + verify(taskExecutorGateway, Mockito.timeout(timeout.toMilliseconds())).disconnectResourceManager(any(TimeoutException.class)); } finally { rpcService.stopService(); @@ -575,7 +582,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); - final TestingSerialRpcService rpcService = new TestingSerialRpcService(); + final TestingRpcService rpcService = new TestingRpcService(); rpcService.registerGateway(jobMasterAddress, jobMasterGateway); final ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration( @@ -620,17 +627,19 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { resourceManager.start(); + final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); + rmLeaderElectionService.isLeader(rmLeaderId); // test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time - CompletableFuture successfulFuture = resourceManager.registerJobManager( + CompletableFuture successfulFuture = rmGateway.registerJobManager( rmLeaderId, jmLeaderId, jmResourceId, jobMasterAddress, jobId, - Time.milliseconds(0L)); - RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS); + timeout); + RegistrationResponse response = successfulFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(response instanceof JobMasterRegistrationSuccess); ArgumentCaptor heartbeatRunnableCaptor = ArgumentCaptor.forClass(Runnable.class); @@ -657,7 +666,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { // run the timeout runnable to simulate a heartbeat timeout timeoutRunnable.run(); - verify(jobMasterGateway).disconnectResourceManager(eq(jmLeaderId), eq(rmLeaderId), any(TimeoutException.class)); + verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).disconnectResourceManager(eq(jmLeaderId), eq(rmLeaderId), any(TimeoutException.class)); } finally { rpcService.stopService(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java index aeceb59e6ef9d..68c43f8af9e76 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java @@ -28,15 +28,15 @@ import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.SlotRequest; -import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; import java.util.List; import java.util.UUID; @@ -51,218 +51,268 @@ import static org.mockito.Matchers.any; import static org.mockito.Mockito.RETURNS_MOCKS; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class SlotPoolTest extends TestLogger { + private final Time timeout = Time.seconds(10L); + private RpcService rpcService; private JobID jobId; - private MainThreadValidatorUtil mainThreadValidatorUtil; - - private SlotPool slotPool; - - private ResourceManagerGateway resourceManagerGateway; - @Before public void setUp() throws Exception { - - this.rpcService = new TestingSerialRpcService(); + this.rpcService = new TestingRpcService(); this.jobId = new JobID(); - this.slotPool = new SlotPool(rpcService, jobId); - - this.mainThreadValidatorUtil = new MainThreadValidatorUtil(slotPool); - - mainThreadValidatorUtil.enterMainThread(); - - final String jobManagerAddress = "foobar"; - - slotPool.start(UUID.randomUUID(), jobManagerAddress); - - this.resourceManagerGateway = mock(ResourceManagerGateway.class); - when(resourceManagerGateway - .requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class))) - .thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); - - slotPool.connectToResourceManager(UUID.randomUUID(), resourceManagerGateway); } @After public void tearDown() throws Exception { - mainThreadValidatorUtil.exitMainThread(); + rpcService.stopService(); } @Test public void testAllocateSimpleSlot() throws Exception { - ResourceID resourceID = new ResourceID("resource"); - slotPool.registerTaskManager(resourceID); - - ScheduledUnit task = mock(ScheduledUnit.class); - CompletableFuture future = slotPool.allocateSlot(task, DEFAULT_TESTING_PROFILE, null, Time.milliseconds(0L)); - assertFalse(future.isDone()); - - ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); - - final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); - - AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); - assertTrue(slotPool.offerSlot(allocatedSlot).get()); - - SimpleSlot slot = future.get(1, TimeUnit.SECONDS); - assertTrue(future.isDone()); - assertTrue(slot.isAlive()); - assertEquals(resourceID, slot.getTaskManagerID()); - assertEquals(jobId, slot.getJobID()); - assertEquals(slotPool.getSlotOwner(), slot.getOwner()); - assertEquals(slotPool.getAllocatedSlots().get(slot.getAllocatedSlot().getSlotAllocationId()), slot); + ResourceManagerGateway resourceManagerGateway = createResourceManagerGatewayMock(); + final SlotPool slotPool = new SlotPool(rpcService, jobId); + + try { + SlotPoolGateway slotPoolGateway = setupSlotPool(slotPool, resourceManagerGateway); + ResourceID resourceID = new ResourceID("resource"); + slotPoolGateway.registerTaskManager(resourceID); + + ScheduledUnit task = mock(ScheduledUnit.class); + CompletableFuture future = slotPoolGateway.allocateSlot(task, DEFAULT_TESTING_PROFILE, null, timeout); + assertFalse(future.isDone()); + + ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + + final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); + + AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); + assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); + + SimpleSlot slot = future.get(1, TimeUnit.SECONDS); + assertTrue(future.isDone()); + assertTrue(slot.isAlive()); + assertEquals(resourceID, slot.getTaskManagerID()); + assertEquals(jobId, slot.getJobID()); + assertEquals(slotPool.getSlotOwner(), slot.getOwner()); + assertEquals(slotPool.getAllocatedSlots().get(slot.getAllocatedSlot().getSlotAllocationId()), slot); + } finally { + slotPool.shutDown(); + } } @Test public void testAllocationFulfilledByReturnedSlot() throws Exception { - ResourceID resourceID = new ResourceID("resource"); - slotPool.registerTaskManager(resourceID); + ResourceManagerGateway resourceManagerGateway = createResourceManagerGatewayMock(); + final SlotPool slotPool = new SlotPool(rpcService, jobId); + + try { + SlotPoolGateway slotPoolGateway = setupSlotPool(slotPool, resourceManagerGateway); + ResourceID resourceID = new ResourceID("resource"); + slotPool.registerTaskManager(resourceID); - CompletableFuture future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null, Time.milliseconds(0L)); - CompletableFuture future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null, Time.milliseconds(0L)); + CompletableFuture future1 = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + CompletableFuture future2 = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); - assertFalse(future1.isDone()); - assertFalse(future2.isDone()); + assertFalse(future1.isDone()); + assertFalse(future2.isDone()); - ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, times(2)) - .requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds()).times(2)) + .requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); - final List slotRequests = slotRequestArgumentCaptor.getAllValues(); + final List slotRequests = slotRequestArgumentCaptor.getAllValues(); - AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequests.get(0).getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); - assertTrue(slotPool.offerSlot(allocatedSlot).get()); + AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequests.get(0).getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); + assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); - SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); - assertTrue(future1.isDone()); - assertFalse(future2.isDone()); + SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); + assertTrue(future1.isDone()); + assertFalse(future2.isDone()); - // return this slot to pool - slot1.releaseSlot(); + // return this slot to pool + slot1.releaseSlot(); - // second allocation fulfilled by previous slot returning - SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS); - assertTrue(future2.isDone()); + // second allocation fulfilled by previous slot returning + SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS); + assertTrue(future2.isDone()); - assertNotEquals(slot1, slot2); - assertTrue(slot1.isReleased()); - assertTrue(slot2.isAlive()); - assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID()); - assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber()); - assertEquals(slotPool.getAllocatedSlots().get(slot1.getAllocatedSlot().getSlotAllocationId()), slot2); + assertNotEquals(slot1, slot2); + assertTrue(slot1.isReleased()); + assertTrue(slot2.isAlive()); + assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID()); + assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber()); + assertEquals(slotPool.getAllocatedSlots().get(slot1.getAllocatedSlot().getSlotAllocationId()), slot2); + } finally { + slotPool.shutDown(); + } } @Test public void testAllocateWithFreeSlot() throws Exception { - ResourceID resourceID = new ResourceID("resource"); - slotPool.registerTaskManager(resourceID); + ResourceManagerGateway resourceManagerGateway = createResourceManagerGatewayMock(); + final SlotPool slotPool = new SlotPool(rpcService, jobId); - CompletableFuture future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null, Time.milliseconds(0L)); - assertFalse(future1.isDone()); + try { + SlotPoolGateway slotPoolGateway = setupSlotPool(slotPool, resourceManagerGateway); + ResourceID resourceID = new ResourceID("resource"); + slotPoolGateway.registerTaskManager(resourceID); - ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + CompletableFuture future1 = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + assertFalse(future1.isDone()); - final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); + ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); - AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); - assertTrue(slotPool.offerSlot(allocatedSlot).get()); + final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); - SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); - assertTrue(future1.isDone()); + AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); + assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); - // return this slot to pool - slot1.releaseSlot(); + SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); + assertTrue(future1.isDone()); - CompletableFuture future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null, Time.milliseconds(0L)); + // return this slot to pool + slot1.releaseSlot(); - // second allocation fulfilled by previous slot returning - SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS); - assertTrue(future2.isDone()); + CompletableFuture future2 = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); - assertNotEquals(slot1, slot2); - assertTrue(slot1.isReleased()); - assertTrue(slot2.isAlive()); - assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID()); - assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber()); + // second allocation fulfilled by previous slot returning + SimpleSlot slot2 = future2.get(1, TimeUnit.SECONDS); + assertTrue(future2.isDone()); + + assertNotEquals(slot1, slot2); + assertTrue(slot1.isReleased()); + assertTrue(slot2.isAlive()); + assertEquals(slot1.getTaskManagerID(), slot2.getTaskManagerID()); + assertEquals(slot1.getSlotNumber(), slot2.getSlotNumber()); + } finally { + slotPool.shutDown(); + } } @Test public void testOfferSlot() throws Exception { - ResourceID resourceID = new ResourceID("resource"); - slotPool.registerTaskManager(resourceID); + ResourceManagerGateway resourceManagerGateway = createResourceManagerGatewayMock(); + final SlotPool slotPool = new SlotPool(rpcService, jobId); + + try { + SlotPoolGateway slotPoolGateway = setupSlotPool(slotPool, resourceManagerGateway); + ResourceID resourceID = new ResourceID("resource"); + slotPoolGateway.registerTaskManager(resourceID); - CompletableFuture future = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null, Time.milliseconds(0L)); - assertFalse(future.isDone()); + CompletableFuture future = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + assertFalse(future.isDone()); - ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); - final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); + final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); - // slot from unregistered resource - AllocatedSlot invalid = createAllocatedSlot(new ResourceID("unregistered"), slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); - assertFalse(slotPool.offerSlot(invalid).get()); + // slot from unregistered resource + AllocatedSlot invalid = createAllocatedSlot(new ResourceID("unregistered"), slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); + assertFalse(slotPoolGateway.offerSlot(invalid).get()); - AllocatedSlot notRequested = createAllocatedSlot(resourceID, new AllocationID(), jobId, DEFAULT_TESTING_PROFILE); + AllocatedSlot notRequested = createAllocatedSlot(resourceID, new AllocationID(), jobId, DEFAULT_TESTING_PROFILE); - // we'll also accept non requested slots - assertTrue(slotPool.offerSlot(notRequested).get()); + // we'll also accept non requested slots + assertTrue(slotPoolGateway.offerSlot(notRequested).get()); - AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); + AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); - // accepted slot - assertTrue(slotPool.offerSlot(allocatedSlot).get()); - SimpleSlot slot = future.get(1, TimeUnit.SECONDS); - assertTrue(future.isDone()); - assertTrue(slot.isAlive()); + // accepted slot + assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); + SimpleSlot slot = future.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + assertTrue(slot.isAlive()); - // duplicated offer with using slot - assertTrue(slotPool.offerSlot(allocatedSlot).get()); - assertTrue(future.isDone()); - assertTrue(slot.isAlive()); + // duplicated offer with using slot + assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); + assertTrue(slot.isAlive()); - // duplicated offer with free slot - slot.releaseSlot(); - assertTrue(slot.isReleased()); - assertTrue(slotPool.offerSlot(allocatedSlot).get()); + // duplicated offer with free slot + slot.releaseSlot(); + assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); + } finally { + slotPool.shutDown(); + } } @Test public void testReleaseResource() throws Exception { - ResourceID resourceID = new ResourceID("resource"); - slotPool.registerTaskManager(resourceID); + ResourceManagerGateway resourceManagerGateway = createResourceManagerGatewayMock(); + + final CompletableFuture slotReturnFuture = new CompletableFuture<>(); + + final SlotPool slotPool = new SlotPool(rpcService, jobId) { + @Override + public void returnAllocatedSlot(Slot slot) { + super.returnAllocatedSlot(slot); + + slotReturnFuture.complete(true); + } + }; + + try { + SlotPoolGateway slotPoolGateway = setupSlotPool(slotPool, resourceManagerGateway); + ResourceID resourceID = new ResourceID("resource"); + slotPoolGateway.registerTaskManager(resourceID); + + CompletableFuture future1 = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + + ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + + final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); + + CompletableFuture future2 = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); + + AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); + assertTrue(slotPoolGateway.offerSlot(allocatedSlot).get()); - CompletableFuture future1 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null, Time.milliseconds(0L)); + SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); + assertTrue(future1.isDone()); + assertFalse(future2.isDone()); - ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + slotPoolGateway.releaseTaskManager(resourceID); - final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); + // wait until the slot has been returned + slotReturnFuture.get(); - CompletableFuture future2 = slotPool.allocateSlot(mock(ScheduledUnit.class),DEFAULT_TESTING_PROFILE, null, Time.milliseconds(0L)); + assertTrue(slot1.isReleased()); - AllocatedSlot allocatedSlot = createAllocatedSlot(resourceID, slotRequest.getAllocationId(), jobId, DEFAULT_TESTING_PROFILE); - assertTrue(slotPool.offerSlot(allocatedSlot).get()); + // slot released and not usable, second allocation still not fulfilled + Thread.sleep(10); + assertFalse(future2.isDone()); + } finally { + slotPool.shutDown(); + } + } + + private static ResourceManagerGateway createResourceManagerGatewayMock() { + ResourceManagerGateway resourceManagerGateway = mock(ResourceManagerGateway.class); + when(resourceManagerGateway + .requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class))) + .thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); - SimpleSlot slot1 = future1.get(1, TimeUnit.SECONDS); - assertTrue(future1.isDone()); - assertFalse(future2.isDone()); + return resourceManagerGateway; + } - slotPool.releaseTaskManager(resourceID); - assertTrue(slot1.isReleased()); + private static SlotPoolGateway setupSlotPool( + SlotPool slotPool, + ResourceManagerGateway resourceManagerGateway) throws Exception { + final String jobManagerAddress = "foobar"; + + slotPool.start(UUID.randomUUID(), jobManagerAddress); + + slotPool.connectToResourceManager(UUID.randomUUID(), resourceManagerGateway); - // slot released and not usable, second allocation still not fulfilled - Thread.sleep(10); - assertFalse(future2.isDone()); + return slotPool.getSelfGateway(SlotPoolGateway.class); } static AllocatedSlot createAllocatedSlot( 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 0c4d3762884e7..df353690761ca 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 @@ -33,8 +33,9 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.TestingFatalErrorHandler; @@ -62,6 +63,8 @@ @PrepareForTest(BlobLibraryCacheManager.class) public class JobMasterTest extends TestLogger { + private final Time testingTimeout = Time.seconds(10L); + @Test public void testHeartbeatTimeoutWithTaskManager() throws Exception { final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); @@ -81,7 +84,7 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(tmResourceId, InetAddress.getLoopbackAddress(), 1234); final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); rpc.registerGateway(taskManagerAddress, taskExecutorGateway); final long heartbeatInterval = 1L; @@ -89,6 +92,8 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { final ScheduledExecutor scheduledExecutor = mock(ScheduledExecutor.class); final HeartbeatServices heartbeatServices = new TestingHeartbeatServices(heartbeatInterval, heartbeatTimeout, scheduledExecutor); + final BlobLibraryCacheManager libraryCacheManager = mock(BlobLibraryCacheManager.class); + when(libraryCacheManager.getBlobServerPort()).thenReturn(1337); final JobGraph jobGraph = new JobGraph(); @@ -101,7 +106,7 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { haServices, heartbeatServices, Executors.newScheduledThreadPool(1), - mock(BlobLibraryCacheManager.class), + libraryCacheManager, mock(RestartStrategyFactory.class), Time.of(10, TimeUnit.SECONDS), null, @@ -111,8 +116,14 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { jobMaster.start(jmLeaderId); + final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); + // register task manager will trigger monitor heartbeat target, schedule heartbeat request at interval time - jobMaster.registerTaskManager(taskManagerAddress, taskManagerLocation, jmLeaderId, Time.milliseconds(0L)); + CompletableFuture registrationResponse = jobMasterGateway + .registerTaskManager(taskManagerAddress, taskManagerLocation, jmLeaderId, testingTimeout); + + // wait for the completion of the registration + registrationResponse.get(); ArgumentCaptor heartbeatRunnableCaptor = ArgumentCaptor.forClass(Runnable.class); verify(scheduledExecutor, times(1)).scheduleAtFixedRate( @@ -124,7 +135,7 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { Runnable heartbeatRunnable = heartbeatRunnableCaptor.getValue(); ArgumentCaptor timeoutRunnableCaptor = ArgumentCaptor.forClass(Runnable.class); - verify(scheduledExecutor).schedule(timeoutRunnableCaptor.capture(), eq(heartbeatTimeout), eq(TimeUnit.MILLISECONDS)); + verify(scheduledExecutor, timeout(testingTimeout.toMilliseconds())).schedule(timeoutRunnableCaptor.capture(), eq(heartbeatTimeout), eq(TimeUnit.MILLISECONDS)); Runnable timeoutRunnable = timeoutRunnableCaptor.getValue(); @@ -136,7 +147,7 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { // run the timeout runnable to simulate a heartbeat timeout timeoutRunnable.run(); - verify(taskExecutorGateway).disconnectJobManager(eq(jobGraph.getJobID()), any(TimeoutException.class)); + verify(taskExecutorGateway, timeout(testingTimeout.toMilliseconds())).disconnectJobManager(eq(jobGraph.getJobID()), any(TimeoutException.class)); // check if a concurrent error occurred testingFatalErrorHandler.rethrowError(); @@ -179,7 +190,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { )).thenReturn(CompletableFuture.completedFuture(new JobMasterRegistrationSuccess( heartbeatInterval, rmLeaderId, rmResourceId))); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); @@ -207,7 +218,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { rmLeaderRetrievalService.notifyListener(resourceManagerAddress, rmLeaderId); // register job manager success will trigger monitor heartbeat target between jm and rm - verify(resourceManagerGateway).registerJobManager( + verify(resourceManagerGateway, timeout(testingTimeout.toMilliseconds())).registerJobManager( eq(rmLeaderId), eq(jmLeaderId), eq(jmResourceId), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index 986f8480aadda..c213752081de3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerConfiguration; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.TestLogger; @@ -35,6 +35,7 @@ import org.junit.Test; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import static org.mockito.Mockito.mock; @@ -46,9 +47,17 @@ public class ResourceManagerHATest extends TestLogger { @Test public void testGrantAndRevokeLeadership() throws Exception { ResourceID rmResourceId = ResourceID.generate(); - RpcService rpcService = new TestingSerialRpcService(); + RpcService rpcService = new TestingRpcService(); + + CompletableFuture leaderSessionIdFuture = new CompletableFuture<>(); + + TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService() { + @Override + public void confirmLeaderSessionID(UUID leaderId) { + leaderSessionIdFuture.complete(leaderId); + } + }; - TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService(); TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService); @@ -73,6 +82,8 @@ public void testGrantAndRevokeLeadership() throws Exception { TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + CompletableFuture revokedLeaderIdFuture = new CompletableFuture<>(); + final ResourceManager resourceManager = new StandaloneResourceManager( rpcService, @@ -84,17 +95,25 @@ public void testGrantAndRevokeLeadership() throws Exception { resourceManagerRuntimeServices.getSlotManager(), metricRegistry, resourceManagerRuntimeServices.getJobLeaderIdService(), - testingFatalErrorHandler); + testingFatalErrorHandler) { + + @Override + public void revokeLeadership() { + super.revokeLeadership(); + runAsync( + () -> revokedLeaderIdFuture.complete(getLeaderSessionId())); + } + }; resourceManager.start(); // before grant leadership, resourceManager's leaderId is null Assert.assertEquals(null, resourceManager.getLeaderSessionId()); final UUID leaderId = UUID.randomUUID(); leaderElectionService.isLeader(leaderId); // after grant leadership, resourceManager's leaderId has value - Assert.assertEquals(leaderId, resourceManager.getLeaderSessionId()); + Assert.assertEquals(leaderId, leaderSessionIdFuture.get()); // then revoke leadership, resourceManager's leaderId is null again leaderElectionService.notLeader(); - Assert.assertEquals(null, resourceManager.getLeaderSessionId()); + Assert.assertEquals(null, revokedLeaderIdFuture.get()); if (testingFatalErrorHandler.hasExceptionOccurred()) { testingFatalErrorHandler.rethrowError(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java index 10d6a72721a86..139bfc4e19e58 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; @@ -50,13 +50,13 @@ public class ResourceManagerJobMasterTest extends TestLogger { - private TestingSerialRpcService rpcService; + private TestingRpcService rpcService; - private final Time timeout = Time.milliseconds(0L); + private final Time timeout = Time.seconds(10L); @Before public void setup() throws Exception { - rpcService = new TestingSerialRpcService(); + rpcService = new TestingRpcService(); } @After @@ -77,17 +77,18 @@ public void testRegisterJobMaster() throws Exception { TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); // test response successful - CompletableFuture successfulFuture = resourceManager.registerJobManager( + CompletableFuture successfulFuture = rmGateway.registerJobManager( rmLeaderSessionId, jmLeaderID, jmResourceId, jobMasterAddress, jobID, timeout); - RegistrationResponse response = successfulFuture.get(5L, TimeUnit.SECONDS); + RegistrationResponse response = successfulFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(response instanceof JobMasterRegistrationSuccess); if (testingFatalErrorHandler.hasExceptionOccurred()) { @@ -108,11 +109,12 @@ public void testRegisterJobMasterWithUnmatchedLeaderSessionId1() throws Exceptio TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); // test throw exception when receive a registration from job master which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); - CompletableFuture unMatchedLeaderFuture = resourceManager.registerJobManager( + CompletableFuture unMatchedLeaderFuture = rmGateway.registerJobManager( differentLeaderSessionID, jmLeaderID, jmResourceId, @@ -139,13 +141,14 @@ public void testRegisterJobMasterWithUnmatchedLeaderSessionId2() throws Exceptio HighAvailabilityServices.DEFAULT_LEADER_ID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); // test throw exception when receive a registration from job master which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); - CompletableFuture unMatchedLeaderFuture = resourceManager.registerJobManager( + CompletableFuture unMatchedLeaderFuture = rmGateway.registerJobManager( rmLeaderSessionId, differentLeaderSessionID, jmResourceId, @@ -172,13 +175,14 @@ public void testRegisterJobMasterFromInvalidAddress() throws Exception { HighAvailabilityServices.DEFAULT_LEADER_ID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); // test throw exception when receive a registration from job master which takes invalid address String invalidAddress = "/jobMasterAddress2"; - CompletableFuture invalidAddressFuture = resourceManager.registerJobManager( + CompletableFuture invalidAddressFuture = rmGateway.registerJobManager( rmLeaderSessionId, jmLeaderSessionId, jmResourceId, @@ -204,21 +208,26 @@ public void testRegisterJobMasterWithFailureLeaderListener() throws Exception { "localhost", HighAvailabilityServices.DEFAULT_LEADER_ID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManager resourceManager = createAndStartResourceManager( + resourceManagerLeaderElectionService, + jobID, + jobMasterLeaderRetrievalService, + testingFatalErrorHandler); + final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); JobID unknownJobIDToHAServices = new JobID(); // verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener - CompletableFuture declineFuture = resourceManager.registerJobManager( + CompletableFuture declineFuture = rmGateway.registerJobManager( rmLeaderSessionId, jmLeaderSessionId, jmResourceId, jobMasterAddress, unknownJobIDToHAServices, timeout); - RegistrationResponse response = declineFuture.get(5, TimeUnit.SECONDS); + RegistrationResponse response = declineFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(response instanceof RegistrationResponse.Decline); if (testingFatalErrorHandler.hasExceptionOccurred()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index fc96f0d27a651..616ed5c985408 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; @@ -49,7 +49,9 @@ public class ResourceManagerTaskExecutorTest extends TestLogger { - private TestingSerialRpcService rpcService; + private final Time timeout = Time.seconds(10L); + + private TestingRpcService rpcService; private SlotReport slotReport = new SlotReport(); @@ -61,19 +63,22 @@ public class ResourceManagerTaskExecutorTest extends TestLogger { private StandaloneResourceManager resourceManager; + private ResourceManagerGateway rmGateway; + private UUID leaderSessionId; private TestingFatalErrorHandler testingFatalErrorHandler; @Before public void setup() throws Exception { - rpcService = new TestingSerialRpcService(); + rpcService = new TestingRpcService(); taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress); resourceManagerResourceID = ResourceID.generate(); TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService(); testingFatalErrorHandler = new TestingFatalErrorHandler(); resourceManager = createAndStartResourceManager(rmLeaderElectionService, testingFatalErrorHandler); + rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); leaderSessionId = grantLeadership(rmLeaderElectionService); } @@ -90,13 +95,13 @@ public void testRegisterTaskExecutor() throws Exception { try { // test response successful CompletableFuture successfulFuture = - resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport, Time.milliseconds(0L)); - RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS); + rmGateway.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); + RegistrationResponse response = successfulFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(response instanceof TaskExecutorRegistrationSuccess); // test response successful with instanceID not equal to previous when receive duplicate registration from taskExecutor CompletableFuture duplicateFuture = - resourceManager.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport, Time.milliseconds(0L)); + rmGateway.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); RegistrationResponse duplicateResponse = duplicateFuture.get(); assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess); assertNotEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId()); @@ -116,8 +121,8 @@ public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Except // test throw exception when receive a registration from taskExecutor which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); CompletableFuture unMatchedLeaderFuture = - resourceManager.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID, slotReport, Time.milliseconds(0L)); - assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); + rmGateway.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); + assertTrue(unMatchedLeaderFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS) instanceof RegistrationResponse.Decline); } finally { if (testingFatalErrorHandler.hasExceptionOccurred()) { testingFatalErrorHandler.rethrowError(); @@ -134,8 +139,8 @@ public void testRegisterTaskExecutorFromInvalidAddress() throws Exception { // test throw exception when receive a registration from taskExecutor which takes invalid address String invalidAddress = "/taskExecutor2"; CompletableFuture invalidAddressFuture = - resourceManager.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID, slotReport, Time.milliseconds(0L)); - assertTrue(invalidAddressFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); + rmGateway.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID, slotReport, timeout); + assertTrue(invalidAddressFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS) instanceof RegistrationResponse.Decline); } finally { if (testingFatalErrorHandler.hasExceptionOccurred()) { testingFatalErrorHandler.rethrowError(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java deleted file mode 100644 index cb38f6fee5612..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java +++ /dev/null @@ -1,440 +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.rpc; - -import org.apache.flink.api.common.time.Time; -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.util.DirectExecutorService; -import org.apache.flink.util.Preconditions; - -import java.lang.annotation.Annotation; -import java.lang.reflect.InvocationHandler; -import java.lang.reflect.Method; -import java.lang.reflect.Proxy; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.Delayed; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; -import java.util.concurrent.Future; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * An RPC Service implementation for testing. This RPC service directly executes all asynchronous - * calls one by one in the calling thread. - */ -public class TestingSerialRpcService implements RpcService { - - private final DirectExecutorService executorService; - private final ScheduledExecutorService scheduledExecutorService; - private final ConcurrentHashMap registeredConnections; - private final CompletableFuture terminationFuture; - - private final ScheduledExecutor scheduledExecutorServiceAdapter; - - public TestingSerialRpcService() { - executorService = new DirectExecutorService(); - scheduledExecutorService = new ScheduledThreadPoolExecutor(1); - this.registeredConnections = new ConcurrentHashMap<>(16); - this.terminationFuture = new CompletableFuture<>(); - - this.scheduledExecutorServiceAdapter = new ScheduledExecutorServiceAdapter(scheduledExecutorService); - } - - @Override - public ScheduledFuture scheduleRunnable(final Runnable runnable, final long delay, final TimeUnit unit) { - try { - unit.sleep(delay); - runnable.run(); - - return new DoneScheduledFuture(null); - } catch (Throwable e) { - throw new RuntimeException(e); - } - } - - @Override - public void execute(Runnable runnable) { - runnable.run(); - } - - @Override - public CompletableFuture execute(Callable callable) { - try { - T result = callable.call(); - - return CompletableFuture.completedFuture(result); - } catch (Exception e) { - return FutureUtils.completedExceptionally(e); - } - } - - @Override - public Executor getExecutor() { - return executorService; - } - - public ScheduledExecutor getScheduledExecutor() { - return scheduledExecutorServiceAdapter; - } - - @Override - public void stopService() { - executorService.shutdown(); - - scheduledExecutorService.shutdown(); - - boolean terminated = false; - - try { - terminated = scheduledExecutorService.awaitTermination(1, TimeUnit.SECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - - if (!terminated) { - List runnables = scheduledExecutorService.shutdownNow(); - - for (Runnable runnable : runnables) { - runnable.run(); - } - } - - registeredConnections.clear(); - terminationFuture.complete(null); - } - - @Override - public CompletableFuture getTerminationFuture() { - return terminationFuture; - } - - @Override - public void stopServer(RpcServer selfGateway) { - registeredConnections.remove(selfGateway.getAddress()); - } - - @Override - public RpcServer startServer(S rpcEndpoint) { - final String address = UUID.randomUUID().toString(); - - InvocationHandler akkaInvocationHandler = new TestingSerialRpcService.TestingSerialInvocationHandler<>(address, rpcEndpoint); - ClassLoader classLoader = getClass().getClassLoader(); - - Set> implementedRpcGateways = RpcUtils.extractImplementedRpcGateways(rpcEndpoint.getClass()); - - implementedRpcGateways.add(RpcServer.class); - - - @SuppressWarnings("unchecked") - RpcServer rpcServer = (RpcServer) Proxy.newProxyInstance( - classLoader, - implementedRpcGateways.toArray(new Class[implementedRpcGateways.size()]), - akkaInvocationHandler); - - // register self - registeredConnections.putIfAbsent(rpcServer.getAddress(), rpcServer); - - return rpcServer; - } - - @Override - public String getAddress() { - return ""; - } - - @Override - public int getPort() { - return -1; - } - - @Override - public CompletableFuture connect(String address, Class clazz) { - RpcGateway gateway = registeredConnections.get(address); - - if (gateway != null) { - if (clazz.isAssignableFrom(gateway.getClass())) { - @SuppressWarnings("unchecked") - C typedGateway = (C) gateway; - return CompletableFuture.completedFuture(typedGateway); - } else { - return FutureUtils.completedExceptionally(new Exception("Gateway registered under " + address + " is not of type " + clazz)); - } - } else { - return FutureUtils.completedExceptionally(new Exception("No gateway registered under " + address + '.')); - } - } - - // ------------------------------------------------------------------------ - // connections - // ------------------------------------------------------------------------ - - public void registerGateway(String address, RpcGateway gateway) { - checkNotNull(address); - checkNotNull(gateway); - - if (registeredConnections.putIfAbsent(address, gateway) != null) { - throw new IllegalStateException("a gateway is already registered under " + address); - } - } - - public void clearGateways() { - registeredConnections.clear(); - } - - private static final class TestingSerialInvocationHandler implements InvocationHandler, RpcGateway, MainThreadExecutable, StartStoppable { - - private final T rpcEndpoint; - - /** default timeout for asks */ - private final Time timeout; - - private final String address; - - private TestingSerialInvocationHandler(String address, T rpcEndpoint) { - this(address, rpcEndpoint, Time.seconds(10)); - } - - private TestingSerialInvocationHandler(String address, T rpcEndpoint, Time timeout) { - this.rpcEndpoint = rpcEndpoint; - this.timeout = timeout; - this.address = address; - } - - @Override - public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { - Class declaringClass = method.getDeclaringClass(); - if (declaringClass.equals(MainThreadExecutable.class) || - declaringClass.equals(Object.class) || - declaringClass.equals(StartStoppable.class) || - declaringClass.equals(RpcServer.class) || - declaringClass.equals(RpcGateway.class)) { - return method.invoke(this, args); - } else { - final String methodName = method.getName(); - Class[] parameterTypes = method.getParameterTypes(); - Annotation[][] parameterAnnotations = method.getParameterAnnotations(); - Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); - - Class returnType = method.getReturnType(); - - if (returnType.equals(CompletableFuture.class)) { - try { - Object result = handleRpcInvocationSync(methodName, parameterTypes, args, futureTimeout); - return CompletableFuture.completedFuture(result); - } catch (Throwable e) { - return FutureUtils.completedExceptionally(e); - } - } else { - return handleRpcInvocationSync(methodName, parameterTypes, args, futureTimeout); - } - } - } - - /** - * Handle rpc invocations by looking up the rpc method on the rpc endpoint and calling this - * method with the provided method arguments. If the method has a return value, it is returned - * to the sender of the call. - */ - private Object handleRpcInvocationSync(final String methodName, - final Class[] parameterTypes, - final Object[] args, - final Time futureTimeout) throws Exception { - final Method rpcMethod = lookupRpcMethod(methodName, parameterTypes); - Object result = rpcMethod.invoke(rpcEndpoint, args); - - if (result instanceof Future) { - Future future = (Future) result; - return future.get(futureTimeout.getSize(), futureTimeout.getUnit()); - } else { - return result; - } - } - - @Override - public void runAsync(Runnable runnable) { - runnable.run(); - } - - @Override - public CompletableFuture callAsync(Callable callable, Time callTimeout) { - try { - return CompletableFuture.completedFuture(callable.call()); - } catch (Throwable e) { - return FutureUtils.completedExceptionally(e); - } - } - - @Override - public void scheduleRunAsync(final Runnable runnable, final long delay) { - try { - TimeUnit.MILLISECONDS.sleep(delay); - runnable.run(); - } catch (Throwable e) { - throw new RuntimeException(e); - } - } - - @Override - public String getAddress() { - return address; - } - - // this is not a real hostname but the address above is also not a real akka RPC address - // and we keep it that way until actually needed by a test case - @Override - public String getHostname() { - return address; - } - - @Override - public void start() { - // do nothing - } - - @Override - public void stop() { - // do nothing - } - - /** - * Look up the rpc method on the given {@link RpcEndpoint} instance. - * - * @param methodName Name of the method - * @param parameterTypes Parameter types of the method - * @return Method of the rpc endpoint - * @throws NoSuchMethodException Thrown if the method with the given name and parameter types - * cannot be found at the rpc endpoint - */ - private Method lookupRpcMethod(final String methodName, - final Class[] parameterTypes) throws NoSuchMethodException { - return rpcEndpoint.getClass().getMethod(methodName, parameterTypes); - } - - // ------------------------------------------------------------------------ - // Helper methods - // ------------------------------------------------------------------------ - - /** - * Extracts the {@link RpcTimeout} annotated rpc timeout value from the list of given method - * arguments. If no {@link RpcTimeout} annotated parameter could be found, then the default - * timeout is returned. - * - * @param parameterAnnotations Parameter annotations - * @param args Array of arguments - * @param defaultTimeout Default timeout to return if no {@link RpcTimeout} annotated parameter - * has been found - * @return Timeout extracted from the array of arguments or the default timeout - */ - private static Time extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, - Time defaultTimeout) { - if (args != null) { - Preconditions.checkArgument(parameterAnnotations.length == args.length); - - for (int i = 0; i < parameterAnnotations.length; i++) { - if (isRpcTimeout(parameterAnnotations[i])) { - if (args[i] instanceof Time) { - return (Time) args[i]; - } else { - throw new RuntimeException("The rpc timeout parameter must be of type " + - Time.class.getName() + ". The type " + args[i].getClass().getName() + - " is not supported."); - } - } - } - } - - return defaultTimeout; - } - - /** - * Checks whether any of the annotations is of type {@link RpcTimeout} - * - * @param annotations Array of annotations - * @return True if {@link RpcTimeout} was found; otherwise false - */ - private static boolean isRpcTimeout(Annotation[] annotations) { - for (Annotation annotation : annotations) { - if (annotation.annotationType().equals(RpcTimeout.class)) { - return true; - } - } - - return false; - } - - } - - private static class DoneScheduledFuture implements ScheduledFuture { - - private final V value; - - private DoneScheduledFuture(V value) { - this.value = value; - } - - @Override - public long getDelay(TimeUnit unit) { - return 0L; - } - - @Override - public int compareTo(Delayed o) { - return 0; - } - - @Override - public boolean cancel(boolean mayInterruptIfRunning) { - return false; - } - - @Override - public boolean isCancelled() { - return false; - } - - @Override - public boolean isDone() { - return true; - } - - @Override - public V get() throws InterruptedException, ExecutionException { - return value; - } - - @Override - public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { - return value; - } - } - -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index 4c87671de2eb9..90f731dc4ae54 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.registration.RegistrationResponse; @@ -47,7 +48,7 @@ import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TimerService; @@ -57,6 +58,7 @@ import org.apache.flink.util.TestLogger; import org.hamcrest.Matchers; import org.junit.Test; +import org.mockito.Mockito; import java.net.InetAddress; import java.util.Arrays; @@ -76,6 +78,8 @@ public class TaskExecutorITCase extends TestLogger { + private final Time timeout = Time.seconds(10L); + @Test public void testSlotAllocation() throws Exception { TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); @@ -98,7 +102,7 @@ public void testSlotAllocation() throws Exception { testingHAServices.setResourceManagerLeaderRetriever(rmLeaderRetrievalService); testingHAServices.setJobMasterLeaderRetriever(jobId, new TestingLeaderRetrievalService(jmAddress, jmLeaderId)); - TestingSerialRpcService rpcService = new TestingSerialRpcService(); + TestingRpcService rpcService = new TestingRpcService(); ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration( Time.milliseconds(500L), Time.milliseconds(500L)); @@ -170,6 +174,7 @@ public void testSlotAllocation() throws Exception { rpcService.registerGateway(rmAddress, resourceManager.getSelfGateway(ResourceManagerGateway.class)); rpcService.registerGateway(jmAddress, jmGateway); + rpcService.registerGateway(taskExecutor.getAddress(), taskExecutor.getSelfGateway(TaskExecutorGateway.class)); final AllocationID allocationId = new AllocationID(); final SlotRequest slotRequest = new SlotRequest(jobId, allocationId, resourceProfile, jmAddress); @@ -179,27 +184,31 @@ public void testSlotAllocation() throws Exception { resourceManager.start(); taskExecutor.start(); + final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); + // notify the RM that it is the leader rmLeaderElectionService.isLeader(rmLeaderId); // notify the TM about the new RM leader rmLeaderRetrievalService.notifyListener(rmAddress, rmLeaderId); - CompletableFuture registrationResponseFuture = resourceManager.registerJobManager( + CompletableFuture registrationResponseFuture = rmGateway.registerJobManager( rmLeaderId, jmLeaderId, jmResourceId, jmAddress, jobId, - Time.milliseconds(0L)); + timeout); RegistrationResponse registrationResponse = registrationResponseFuture.get(); assertTrue(registrationResponse instanceof JobMasterRegistrationSuccess); - resourceManager.requestSlot(jmLeaderId, rmLeaderId, slotRequest, Time.milliseconds(0L)); + CompletableFuture slotAck = rmGateway.requestSlot(jmLeaderId, rmLeaderId, slotRequest, timeout); + + slotAck.get(); - verify(jmGateway).offerSlots( + verify(jmGateway, Mockito.timeout(timeout.toMilliseconds())).offerSlots( eq(taskManagerResourceId), (Iterable)argThat(Matchers.contains(slotOffer)), eq(jmLeaderId), any(Time.class)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index b5a3c8025a463..43ff60bd90d0e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -65,7 +65,6 @@ import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.TestingRpcService; -import org.apache.flink.runtime.rpc.TestingSerialRpcService; import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; @@ -84,6 +83,7 @@ import org.junit.rules.TestName; import org.mockito.ArgumentCaptor; import org.mockito.Matchers; +import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; @@ -123,7 +123,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(tmResourceId, InetAddress.getLoopbackAddress(), 1234); final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(mock(ResourceProfile.class)), mock(TimerService.class)); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation); final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); final TestingLeaderRetrievalService rmLeaderRetrievalService = new TestingLeaderRetrievalService( @@ -205,7 +205,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro jmLeaderRetrievalService.notifyListener(jobMasterAddress, jmLeaderId); // register task manager success will trigger monitoring heartbeat target between tm and jm - verify(jobMasterGateway).registerTaskManager( + verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).registerTaskManager( eq(taskManager.getAddress()), eq(taskManagerLocation), eq(jmLeaderId), any(Time.class)); // the timeout should trigger disconnecting from the JobManager @@ -238,7 +238,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { rmResourceId, 10L))); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); rpc.registerGateway(rmAddress, rmGateway); final TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService( @@ -305,7 +305,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation testLeaderService.notifyListener(rmAddress, rmLeaderId); // register resource manager success will trigger monitoring heartbeat target between tm and rm - verify(rmGateway, atLeast(1)).registerTaskExecutor( + verify(rmGateway, Mockito.timeout(timeout.toMilliseconds()).atLeast(1)).registerTaskExecutor( eq(rmLeaderId), eq(taskManager.getAddress()), eq(tmResourceId), any(SlotReport.class), any(Time.class)); // heartbeat timeout should trigger disconnect TaskManager from ResourceManager @@ -463,7 +463,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress); final String jobManagerAddress = "localhost"; - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); try { // register a mock resource manager gateway ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); @@ -511,7 +511,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { taskManager.start(); String taskManagerAddress = taskManager.getAddress(); - verify(rmGateway).registerTaskExecutor( + verify(rmGateway, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( any(UUID.class), eq(taskManagerAddress), eq(resourceID), eq(slotReport), any(Time.class)); // check if a concurrent error occurred @@ -533,7 +533,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { final ResourceID rmResourceId1 = new ResourceID(address1); final ResourceID rmResourceId2 = new ResourceID(address2); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); try { // register the mock resource manager gateways ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class); @@ -600,7 +600,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { // define a leader and see that a registration happens testLeaderService.notifyListener(address1, leaderId1); - verify(rmGateway1).registerTaskExecutor( + verify(rmGateway1, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( eq(leaderId1), eq(taskManagerAddress), eq(tmResourceID), any(SlotReport.class), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); @@ -610,7 +610,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { // set a new leader, see that a registration happens testLeaderService.notifyListener(address2, leaderId2); - verify(rmGateway2).registerTaskExecutor( + verify(rmGateway2, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( eq(leaderId2), eq(taskManagerAddress), eq(tmResourceID), eq(slotReport), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); @@ -625,11 +625,11 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { /** * Tests that we can submit a task to the TaskManager given that we've allocated a slot there. */ - @Test(timeout = 1000L) + @Test(timeout = 10000L) public void testTaskSubmission() throws Exception { final Configuration configuration = new Configuration(); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final JobID jobId = new JobID(); final AllocationID allocationId = new AllocationID(); @@ -728,7 +728,9 @@ public void testTaskSubmission() throws Exception { taskManager.start(); - taskManager.submitTask(tdd, jobManagerLeaderId, timeout); + final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); + + tmGateway.submitTask(tdd, jobManagerLeaderId, timeout); CompletableFuture completionFuture = TestInvokable.completableFuture; @@ -762,7 +764,7 @@ public void invoke() throws Exception { public void testJobLeaderDetection() throws Exception { final JobID jobId = new JobID(); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); final Configuration configuration = new Configuration(); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final ResourceID resourceId = new ResourceID("foobar"); @@ -845,17 +847,27 @@ public void testJobLeaderDetection() throws Exception { taskManager.start(); + final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); + // tell the task manager about the rm leader resourceManagerLeaderRetrievalService.notifyListener(resourceManagerAddress, resourceManagerLeaderId); // request slots from the task manager under the given allocation id - taskManager.requestSlot(slotId, jobId, allocationId, jobManagerAddress, resourceManagerLeaderId, timeout); + CompletableFuture slotRequestAck = tmGateway.requestSlot( + slotId, + jobId, + allocationId, + jobManagerAddress, + resourceManagerLeaderId, + timeout); + + slotRequestAck.get(); // now inform the task manager about the new job leader jobManagerLeaderRetrievalService.notifyListener(jobManagerAddress, jobManagerLeaderId); // the job leader should get the allocation id offered - verify(jobMasterGateway).offerSlots( + verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).offerSlots( any(ResourceID.class), (Iterable)Matchers.argThat(contains(slotOffer)), eq(jobManagerLeaderId), @@ -876,7 +888,7 @@ public void testJobLeaderDetection() throws Exception { public void testSlotAcceptance() throws Exception { final JobID jobId = new JobID(); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); final Configuration configuration = new Configuration(); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final ResourceID resourceId = new ResourceID("foobar"); @@ -963,7 +975,7 @@ public void testSlotAcceptance() throws Exception { // been properly started. jobLeaderService.addJob(jobId, jobManagerAddress); - verify(resourceManagerGateway).notifySlotAvailable( + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).notifySlotAvailable( eq(resourceManagerLeaderId), eq(registrationId), eq(new SlotID(resourceId, 1)), @@ -996,14 +1008,14 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { final JobID jobId = new JobID(); final String jobManagerAddress = "foobar"; - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); try { // register the mock resource manager gateways ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class); rpc.registerGateway(address1, rmGateway1); TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService( - "localhost", + address1, HighAvailabilityServices.DEFAULT_LEADER_ID); TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); @@ -1016,6 +1028,12 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { when(taskManagerLocation.getResourceID()).thenReturn(resourceID); final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class); + when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(new SlotReport()); + when(taskSlotTable.getCurrentAllocation(1)).thenReturn(new AllocationID()); + + when(rmGateway1.registerTaskExecutor(any(UUID.class), anyString(), eq(resourceID), any(SlotReport.class), any(Time.class))).thenReturn( + CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(new InstanceID(), ResourceID.generate(), 1000L))); TaskExecutor taskManager = new TaskExecutor( rpc, @@ -1030,13 +1048,16 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { mock(TaskManagerMetricGroup.class), mock(BroadcastVariableManager.class), mock(FileCache.class), - mock(TaskSlotTable.class), + taskSlotTable, mock(JobManagerTable.class), mock(JobLeaderService.class), testingFatalErrorHandler); taskManager.start(); - String taskManagerAddress = taskManager.getAddress(); + + final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); + + String taskManagerAddress = tmGateway.getAddress(); // no connection initially, since there is no leader assertNull(taskManager.getResourceManagerConnection()); @@ -1044,19 +1065,19 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { // define a leader and see that a registration happens testLeaderService.notifyListener(address1, leaderId); - verify(rmGateway1).registerTaskExecutor( + verify(rmGateway1, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( eq(leaderId), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); // test that allocating a slot works final SlotID slotID = new SlotID(resourceID, 0); - taskManager.requestSlot(slotID, jobId, new AllocationID(), jobManagerAddress, leaderId, timeout); + tmGateway.requestSlot(slotID, jobId, new AllocationID(), jobManagerAddress, leaderId, timeout); // TODO: Figure out the concrete allocation behaviour between RM and TM. Maybe we don't need the SlotID... // test that we can't allocate slots which are blacklisted due to pending confirmation of the RM final SlotID unconfirmedFreeSlotID = new SlotID(resourceID, 1); - CompletableFuture requestSlotFuture = taskManager.requestSlot( + CompletableFuture requestSlotFuture = tmGateway.requestSlot( unconfirmedFreeSlotID, jobId, new AllocationID(), @@ -1073,13 +1094,13 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { } // re-register - verify(rmGateway1).registerTaskExecutor( + verify(rmGateway1, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( eq(leaderId), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class)); testLeaderService.notifyListener(address1, leaderId); // now we should be successful because the slots status has been synced // test that we can't allocate slots which are blacklisted due to pending confirmation of the RM - taskManager.requestSlot( + tmGateway.requestSlot( unconfirmedFreeSlotID, jobId, new AllocationID(), @@ -1103,7 +1124,7 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { public void testSubmitTaskBeforeAcceptSlot() throws Exception { final JobID jobId = new JobID(); - final TestingSerialRpcService rpc = new TestingSerialRpcService(); + final TestingRpcService rpc = new TestingRpcService(); final Configuration configuration = new Configuration(); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final ResourceID resourceId = new ResourceID("foobar"); @@ -1186,13 +1207,15 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { anyString(), anyInt(), anyInt()) ).thenReturn(taskMetricGroup); + final NetworkEnvironment networkMock = mock(NetworkEnvironment.class, Mockito.RETURNS_MOCKS); + final TaskExecutor taskManager = new TaskExecutor( rpc, taskManagerConfiguration, taskManagerLocation, mock(MemoryManager.class), mock(IOManager.class), - mock(NetworkEnvironment.class), + networkMock, haServices, mock(HeartbeatServices.class, RETURNS_MOCKS), mock(MetricRegistry.class), @@ -1204,6 +1227,9 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { jobLeaderService, testingFatalErrorHandler); taskManager.start(); + + final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); + taskSlotTable.allocateSlot(0, jobId, allocationId1, Time.milliseconds(10000L)); taskSlotTable.allocateSlot(1, jobId, allocationId2, Time.milliseconds(10000L)); @@ -1255,15 +1281,15 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { // been properly started. This will also offer the slots to the job master jobLeaderService.addJob(jobId, jobManagerAddress); - verify(jobMasterGateway).offerSlots(any(ResourceID.class), any(Iterable.class), eq(jobManagerLeaderId), any(Time.class)); + verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).offerSlots(any(ResourceID.class), any(Iterable.class), eq(jobManagerLeaderId), any(Time.class)); // submit the task without having acknowledge the offered slots - taskManager.submitTask(tdd, jobManagerLeaderId, timeout); + tmGateway.submitTask(tdd, jobManagerLeaderId, timeout); // acknowledge the offered slots offerResultFuture.complete(Collections.singleton(offer1)); - verify(resourceManagerGateway).notifySlotAvailable( + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).notifySlotAvailable( eq(resourceManagerLeaderId), eq(registrationId), eq(new SlotID(resourceId, 1)), From 98c06addbb3246eb475296c0fb64eee3ef3f442c Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Mon, 14 Aug 2017 16:44:16 +0800 Subject: [PATCH 009/129] [FLINK-7441] [table] Double quote string literals is not supported in Table API and SQL This closes #4538. --- .../org/apache/flink/table/codegen/CodeGenerator.scala | 4 +++- .../expressions/UserDefinedScalarFunctionTest.scala | 10 ++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) 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 63fd058b980cb..28fea59956fd4 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 @@ -26,6 +26,7 @@ import org.apache.calcite.sql.SqlOperator import org.apache.calcite.sql.`type`.SqlTypeName._ import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} import org.apache.calcite.sql.fun.SqlStdOperatorTable._ +import org.apache.commons.lang3.StringEscapeUtils import org.apache.flink.api.common.functions._ import org.apache.flink.api.common.typeinfo._ import org.apache.flink.api.common.typeutils.CompositeType @@ -669,7 +670,8 @@ abstract class CodeGenerator( generateNonNullLiteral(resultType, decimalField) case VARCHAR | CHAR => - generateNonNullLiteral(resultType, "\"" + value.toString + "\"") + val escapedValue = StringEscapeUtils.ESCAPE_JAVA.translate(value.toString) + generateNonNullLiteral(resultType, "\"" + escapedValue + "\"") case SYMBOL => generateSymbol(value.asInstanceOf[Enum[_]]) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala index 9b3407e5c12ef..71ff70d1d04ce 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala @@ -118,6 +118,16 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { "-1") } + @Test + def testDoubleQuoteParameters(): Unit = { + val hello = "\"\"" + testAllApis( + Func3(42, hello), + s"Func3(42, '$hello')", + s"Func3(42, '$hello')", + s"42 and $hello") + } + @Test def testResults(): Unit = { testAllApis( From e30eb13c4a84517ffa502b2c9c7da4b5b8541c9c Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 9 Aug 2017 15:24:35 +0200 Subject: [PATCH 010/129] [FLINK-7399] [checkstyle] Forbid imports from org.codehaus.jackson This closes #4505. --- tools/maven/checkstyle.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml index 3f78054d4471a..79102fac31790 100644 --- a/tools/maven/checkstyle.xml +++ b/tools/maven/checkstyle.xml @@ -211,7 +211,7 @@ This file is based on the checkstyle file of Apache Beam. - + From 8dfb9d00653271ea4adbeb752da8f62d7647b6d8 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 1 Aug 2017 14:58:03 +0200 Subject: [PATCH 011/129] [FLINK-6982] [guava] Integrate flink-shaded-guava-18 This closes #4503. --- .../flink-connector-elasticsearch2/pom.xml | 31 +++++++++++++++++++ .../kafka/KafkaProducerTestBase.java | 4 +-- .../flink-connector-twitter/pom.xml | 11 +++++++ flink-connectors/flink-hbase/pom.xml | 8 ----- flink-connectors/flink-hcatalog/pom.xml | 26 ++++++++++++++++ flink-contrib/flink-storm-examples/pom.xml | 5 +++ .../flink/storm/join/SingleJoinITCase.java | 2 +- flink-libraries/flink-cep/pom.xml | 9 +++--- .../java/org/apache/flink/cep/nfa/NFA.java | 4 +-- .../flink/cep/nfa/compiler/NFACompiler.java | 4 +-- .../org/apache/flink/cep/nfa/GroupITCase.java | 3 +- .../cep/nfa/IterativeConditionsITCase.java | 3 +- .../org/apache/flink/cep/nfa/NFAITCase.java | 3 +- .../flink/cep/nfa/NFATestUtilities.java | 3 +- .../flink/cep/nfa/NotPatternITCase.java | 3 +- .../flink/cep/nfa/SameElementITCase.java | 5 +-- .../flink/cep/nfa/TimesRangeITCase.java | 3 +- .../flink/cep/nfa/UntilConditionITCase.java | 3 +- .../cep/nfa/compiler/NFACompilerTest.java | 9 +++--- .../flink/cep/operator/CEPOperatorTest.java | 3 +- flink-libraries/flink-table/pom.xml | 3 +- flink-optimizer/pom.xml | 5 ++- .../flink/optimizer/dag/SingleInputNode.java | 2 +- .../flink/optimizer/dag/TwoInputNode.java | 2 +- flink-runtime-web/pom.xml | 9 +++--- .../webmonitor/BackPressureStatsTracker.java | 7 +++-- .../StackTraceSampleCoordinator.java | 3 +- .../checkpoints/CheckpointStatsCache.java | 4 +-- .../handlers/JarAccessDeniedHandlerTest.java | 3 +- .../handlers/JobCancellationHandlerTest.java | 3 +- .../handlers/JobDetailsHandlerTest.java | 3 +- .../handlers/JobStoppingHandlerTest.java | 3 +- .../handlers/TaskManagersHandlerTest.java | 3 +- flink-runtime/pom.xml | 10 +++--- .../apache/flink/runtime/blob/BlobUtils.java | 1 + .../runtime/blob/FileSystemBlobStore.java | 4 +-- .../io/network/TaskEventDispatcher.java | 3 +- .../io/network/api/TaskEventHandler.java | 5 +-- .../runtime/io/network/netty/NettyServer.java | 2 +- .../netty/PartitionRequestClientHandler.java | 2 +- .../network/netty/PartitionRequestQueue.java | 2 +- .../partition/ResultPartitionManager.java | 8 +++-- .../partition/consumer/UnionInputGate.java | 5 +-- .../runtime/query/netty/KvStateClient.java | 2 +- .../runtime/query/netty/KvStateServer.java | 2 +- .../flink/runtime/security/SecurityUtils.java | 3 +- .../checkpoint/CheckpointCoordinatorTest.java | 6 ++-- .../FileCacheDeleteValidationTest.java | 5 +-- .../network/buffer/LocalBufferPoolTest.java | 4 ++- .../SpilledSubpartitionViewTest.java | 4 ++- .../consumer/LocalInputChannelTest.java | 4 ++- .../consumer/RemoteInputChannelTest.java | 4 ++- .../util/TestPooledBufferProvider.java | 3 +- .../concurrent/BlockingBackChannelTest.java | 4 +-- .../iterative/concurrent/BrokerTest.java | 4 +-- .../jobmanager/JobManagerStartupTest.java | 2 +- .../ScheduleOrUpdateConsumersTest.java | 4 +-- .../operators/AbstractOuterJoinTaskTest.java | 4 ++- .../operators/LeftOuterJoinTaskTest.java | 4 ++- .../operators/RightOuterJoinTaskTest.java | 4 ++- .../hash/InPlaceMutableHashTableTest.java | 4 ++- .../runtime/state/StateBackendTestBase.java | 4 ++- .../runtime/testingUtils/TestingUtils.scala | 4 +-- .../flink-shaded-curator-recipes/pom.xml | 17 +++++----- .../flink-shaded-curator-test/pom.xml | 9 ++++++ flink-streaming-java/pom.xml | 11 +++---- .../api/graph/StreamGraphHasherV1.java | 7 +++-- .../streaming/api/datastream/SplitStream.java | 2 +- .../api/graph/StreamGraphHasherV2.java | 7 +++-- .../CoFeedbackTransformation.java | 2 +- .../FeedbackTransformation.java | 2 +- .../OneInputTransformation.java | 2 +- .../PartitionTransformation.java | 2 +- .../transformations/SelectTransformation.java | 2 +- .../SideOutputTransformation.java | 2 +- .../transformations/SinkTransformation.java | 2 +- .../transformations/SplitTransformation.java | 2 +- .../TwoInputTransformation.java | 2 +- .../transformations/UnionTransformation.java | 2 +- .../api/windowing/evictors/DeltaEvictor.java | 2 +- .../windowing/EvictingWindowOperator.java | 6 ++-- .../api/AggregationFunctionTest.java | 3 +- .../ContinuousEventTimeTriggerTest.java | 3 +- .../operators/windowing/CountTriggerTest.java | 3 +- .../EventTimeSessionWindowsTest.java | 3 +- .../windowing/EventTimeTriggerTest.java | 3 +- .../windowing/MergingWindowSetTest.java | 3 +- .../ProcessingTimeSessionWindowsTest.java | 3 +- .../windowing/ProcessingTimeTriggerTest.java | 3 +- .../windowing/WindowOperatorTest.java | 5 +-- .../flink/streaming/util/TestHarnessUtil.java | 3 +- flink-tests/pom.xml | 5 +++ .../test/checkpointing/SavepointITCase.java | 6 ++-- .../UdfStreamOperatorCheckpointingITCase.java | 3 +- pom.xml | 18 ++++------- tools/maven/checkstyle.xml | 2 +- tools/maven/suppressions.xml | 9 ++++++ 97 files changed, 302 insertions(+), 163 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch2/pom.xml b/flink-connectors/flink-connector-elasticsearch2/pom.xml index 1f342bc2b2044..1dbe1144cfae2 100644 --- a/flink-connectors/flink-connector-elasticsearch2/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch2/pom.xml @@ -91,4 +91,35 @@ under the License. + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + + + + com.google.guava:guava + + + + + com.google + org.apache.flink.elasticsearch.shaded.com.google + + com.google.protobuf.** + com.google.inject.** + + + + + + + + + + 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 4a611039af4bf..000de5268fe22 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 @@ -45,13 +45,13 @@ import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Preconditions; -import com.google.common.collect.ImmutableSet; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.Test; import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -285,7 +285,7 @@ public int partition(Integer record, byte[] key, byte[] value, String targetTopi properties, topic, partition, - ImmutableSet.copyOf(getIntegersSequence(BrokerRestartingMapper.numElementsBeforeSnapshot)), + Collections.unmodifiableSet(new HashSet<>(getIntegersSequence(BrokerRestartingMapper.numElementsBeforeSnapshot))), 30000L); deleteTestTopic(topic); diff --git a/flink-connectors/flink-connector-twitter/pom.xml b/flink-connectors/flink-connector-twitter/pom.xml index 0f1e44a5d5dcc..2d8d62a39593c 100644 --- a/flink-connectors/flink-connector-twitter/pom.xml +++ b/flink-connectors/flink-connector-twitter/pom.xml @@ -77,12 +77,23 @@ under the License. + com.google.guava:guava com.twitter:hbc-core com.twitter:joauth org.apache.httpcomponents:httpclient org.apache.httpcomponents:httpcore + + + com.google + org.apache.flink.twitter.shaded.com.google + + com.google.protobuf.** + com.google.inject.** + + + diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml index b900d23ded2b3..e18fe39280c8b 100644 --- a/flink-connectors/flink-hbase/pom.xml +++ b/flink-connectors/flink-hbase/pom.xml @@ -127,14 +127,6 @@ under the License. flink-streaming-java_${scala.binary.version} ${project.version} provided - - - - - com.google.guava - guava - - diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml index 10ca36d8acaf2..1e77d7df443f4 100644 --- a/flink-connectors/flink-hcatalog/pom.xml +++ b/flink-connectors/flink-hcatalog/pom.xml @@ -70,6 +70,32 @@ under the License. + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + + + + com.google.guava:guava + + + + + com.google + org.apache.flink.hcatalog.shaded.com.google + + com.google.protobuf.** + com.google.inject.** + + + + + + + net.alchim31.maven diff --git a/flink-contrib/flink-storm-examples/pom.xml b/flink-contrib/flink-storm-examples/pom.xml index cddb6ef5f05bc..b85e402b35402 100644 --- a/flink-contrib/flink-storm-examples/pom.xml +++ b/flink-contrib/flink-storm-examples/pom.xml @@ -87,6 +87,11 @@ under the License. + + org.apache.flink + flink-shaded-guava + + org.apache.flink flink-test-utils_${scala.binary.version} diff --git a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/join/SingleJoinITCase.java b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/join/SingleJoinITCase.java index 83531bab38a2e..5d406dba72a7f 100644 --- a/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/join/SingleJoinITCase.java +++ b/flink-contrib/flink-storm-examples/src/test/java/org/apache/flink/storm/join/SingleJoinITCase.java @@ -20,7 +20,7 @@ import org.apache.flink.streaming.util.StreamingProgramTestBase; -import com.google.common.base.Joiner; +import org.apache.flink.shaded.guava18.com.google.common.base.Joiner; /** * Test for the SingleJoin example. diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml index 35045c03d9c19..23978b2f72012 100644 --- a/flink-libraries/flink-cep/pom.xml +++ b/flink-libraries/flink-cep/pom.xml @@ -52,11 +52,10 @@ under the License. provided - - com.google.guava - guava - ${guava.version} - + + org.apache.flink + flink-shaded-guava + diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java index 2f6f02eb9bfb7..23084eeb12525 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java @@ -42,8 +42,8 @@ import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.util.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterators; +import org.apache.flink.shaded.guava18.com.google.common.base.Predicate; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators; import javax.annotation.Nullable; diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java index 62464d1d4b7ae..9dc1837ea4de0 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java @@ -36,8 +36,8 @@ import org.apache.flink.cep.pattern.conditions.NotCondition; import org.apache.flink.streaming.api.windowing.time.Time; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterators; +import org.apache.flink.shaded.guava18.com.google.common.base.Predicate; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators; import javax.annotation.Nullable; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java index 226a0916ddf21..c2c7cdacf3eb8 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/GroupITCase.java @@ -26,7 +26,8 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import java.util.ArrayList; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java index 910907fa42f9b..80754b7680b64 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/IterativeConditionsITCase.java @@ -27,7 +27,8 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import java.util.ArrayList; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java index a83eb12874279..b7798c58f2c3e 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java @@ -29,7 +29,8 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Assert; import org.junit.Test; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java index 7bf0767755418..7a522ef1bbf7f 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java @@ -21,7 +21,6 @@ import org.apache.flink.cep.Event; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import com.google.common.primitives.Doubles; import org.junit.Assert; import java.util.ArrayList; @@ -96,7 +95,7 @@ private static class EventComparator implements Comparator { @Override public int compare(Event o1, Event o2) { int nameComp = o1.getName().compareTo(o2.getName()); - int priceComp = Doubles.compare(o1.getPrice(), o2.getPrice()); + int priceComp = Double.compare(o1.getPrice(), o2.getPrice()); int idComp = Integer.compare(o1.getId(), o2.getId()); if (nameComp == 0) { if (priceComp == 0) { diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java index 3b95eb4b6064b..9198ff8f07578 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NotPatternITCase.java @@ -25,7 +25,8 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import java.util.ArrayList; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java index 183cb6d13d714..357107fce9cc8 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SameElementITCase.java @@ -26,8 +26,9 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterators; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import java.util.ArrayList; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java index 37a953407cfd6..76ed26ab5965e 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/TimesRangeITCase.java @@ -25,7 +25,8 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import java.util.ArrayList; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java index 639541d1da7ff..f88e5b21ab401 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/UntilConditionITCase.java @@ -25,7 +25,8 @@ import org.apache.flink.cep.pattern.conditions.SimpleCondition; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import java.util.ArrayList; diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java index 6d4329a873a3a..13dc6da927fb6 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java @@ -33,6 +33,8 @@ import org.apache.flink.cep.pattern.conditions.SimpleCondition; import org.apache.flink.util.TestLogger; +import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -42,7 +44,6 @@ import java.util.Map; import java.util.Set; -import static com.google.common.collect.Sets.newHashSet; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -142,14 +143,14 @@ public void testNFACompilerWithSimplePattern() { State startState = stateMap.get("start"); assertTrue(startState.isStart()); final Set> startTransitions = unfoldTransitions(startState); - assertEquals(newHashSet( + assertEquals(Sets.newHashSet( Tuple2.of("middle", StateTransitionAction.TAKE) ), startTransitions); assertTrue(stateMap.containsKey("middle")); State middleState = stateMap.get("middle"); final Set> middleTransitions = unfoldTransitions(middleState); - assertEquals(newHashSet( + assertEquals(Sets.newHashSet( Tuple2.of("middle", StateTransitionAction.IGNORE), Tuple2.of("end", StateTransitionAction.TAKE) ), middleTransitions); @@ -157,7 +158,7 @@ public void testNFACompilerWithSimplePattern() { assertTrue(stateMap.containsKey("end")); State endState = stateMap.get("end"); final Set> endTransitions = unfoldTransitions(endState); - assertEquals(newHashSet( + assertEquals(Sets.newHashSet( Tuple2.of(NFACompiler.ENDING_STATE_NAME, StateTransitionAction.TAKE) ), endTransitions); diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java index 46ad7bed2b46b..8cf67ad35dfd8 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java @@ -43,7 +43,8 @@ import org.apache.flink.types.Either; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.After; import org.junit.Assert; import org.junit.Rule; diff --git a/flink-libraries/flink-table/pom.xml b/flink-libraries/flink-table/pom.xml index 0e943adf7679d..73629135148c6 100644 --- a/flink-libraries/flink-table/pom.xml +++ b/flink-libraries/flink-table/pom.xml @@ -218,8 +218,9 @@ under the License. org.apache.calcite:* org.apache.calcite.avatica:* net.hydromatic:* - org.reflections:* + org.reflections:* org.codehaus.janino:* + com.google.guava:guava diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml index b94e11e7f8acb..a507edecd0c59 100644 --- a/flink-optimizer/pom.xml +++ b/flink-optimizer/pom.xml @@ -57,9 +57,8 @@ under the License. - com.google.guava - guava - ${guava.version} + org.apache.flink + flink-shaded-guava diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java index 5691d194ec152..964e2d609b9fe 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/SingleInputNode.java @@ -55,7 +55,7 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.util.Visitor; -import com.google.common.collect.Sets; +import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; /** * A node in the optimizer's program representation for an operation with a single input. diff --git a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java index a4199a80fdb72..48815dc9a1c3b 100644 --- a/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java +++ b/flink-optimizer/src/main/java/org/apache/flink/optimizer/dag/TwoInputNode.java @@ -58,7 +58,7 @@ import org.apache.flink.runtime.operators.shipping.ShipStrategyType; import org.apache.flink.util.Visitor; -import com.google.common.collect.Sets; +import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; /** * A node in the optimizer plan that represents an operator with a two different inputs, such as Join, diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml index cf8f30784d09c..14c91c5fd8776 100644 --- a/flink-runtime-web/pom.xml +++ b/flink-runtime-web/pom.xml @@ -61,14 +61,13 @@ under the License. - org.javassist - javassist + org.apache.flink + flink-shaded-guava - com.google.guava - guava - ${guava.version} + org.javassist + javassist - - com.google.guava - guava - ${guava.version} - - org.scala-lang scala-library diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java index 9a13412c90b84..9b5724b084518 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java @@ -26,6 +26,7 @@ import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.util.StringUtils; + import org.slf4j.Logger; import javax.annotation.Nonnull; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java index 83abecb478b16..062fd8273bce5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java @@ -18,13 +18,13 @@ package org.apache.flink.runtime.blob; -import com.google.common.io.Files; - import org.apache.flink.api.common.JobID; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.util.IOUtils; +import org.apache.flink.shaded.guava18.com.google.common.io.Files; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 eddba8db46d42..8816e32cb27e6 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 @@ -18,7 +18,6 @@ package org.apache.flink.runtime.io.network; -import com.google.common.collect.Maps; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -26,6 +25,8 @@ import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; + import java.util.Map; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/TaskEventHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/TaskEventHandler.java index d2dc46b5520b6..9ef170a47795f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/TaskEventHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/TaskEventHandler.java @@ -18,11 +18,12 @@ package org.apache.flink.runtime.io.network.api; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.shaded.guava18.com.google.common.collect.HashMultimap; +import org.apache.flink.shaded.guava18.com.google.common.collect.Multimap; + /** * The event handler manages {@link EventListener} instances and allows to * to publish events to them. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java index e8727d20441b4..4036e2924d202 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyServer.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.util.FatalExitExceptionHandler; +import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer; @@ -32,7 +33,6 @@ import org.apache.flink.shaded.netty4.io.netty.channel.socket.nio.NioServerSocketChannel; import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java index 7db7ac4fe996e..e3097ba1e2eb4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java @@ -31,10 +31,10 @@ import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter; -import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 881eae8312ed0..ff0f1307dbfc7 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 @@ -26,13 +26,13 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputChannel.BufferAndAvailability; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; +import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFutureListener; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter; -import com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java index f681548263491..92fb2a03201db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java @@ -18,11 +18,13 @@ package org.apache.flink.runtime.io.network.partition; -import com.google.common.collect.HashBasedTable; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Table; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; + +import org.apache.flink.shaded.guava18.com.google.common.collect.HashBasedTable; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableList; +import org.apache.flink.shaded.guava18.com.google.common.collect.Table; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 55c78af07d487..87443d261cf2f 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 @@ -18,11 +18,12 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; + import java.io.IOException; import java.util.ArrayDeque; import java.util.Map; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java index 60099d2a35b1a..1a84e831959fe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateClient.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer; import org.apache.flink.util.Preconditions; +import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.flink.shaded.netty4.io.netty.bootstrap.Bootstrap; import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; @@ -39,7 +40,6 @@ import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler; import akka.dispatch.Futures; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.nio.channels.ClosedChannelException; import java.util.ArrayDeque; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java index 2889e2e21aedf..7cf2148273afe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/netty/KvStateServer.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequest; import org.apache.flink.util.Preconditions; +import org.apache.flink.shaded.guava18.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.flink.shaded.netty4.io.netty.bootstrap.ServerBootstrap; import org.apache.flink.shaded.netty4.io.netty.channel.Channel; import org.apache.flink.shaded.netty4.io.netty.channel.ChannelInitializer; @@ -35,7 +36,6 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.LengthFieldBasedFrameDecoder; import org.apache.flink.shaded.netty4.io.netty.handler.stream.ChunkedWriteHandler; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityUtils.java index 9e6f40258a09f..bdaaed6650d8e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityUtils.java @@ -28,7 +28,8 @@ import org.apache.flink.runtime.security.modules.SecurityModule; import org.apache.flink.runtime.security.modules.ZooKeeperModule; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index e78152aef9fff..cb92df66b81bc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.checkpoint; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; @@ -55,6 +53,10 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java index 89ab975dc8903..0782e207627e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/filecache/FileCacheDeleteValidationTest.java @@ -22,8 +22,9 @@ import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry; import org.apache.flink.core.fs.Path; -import com.google.common.base.Charsets; -import com.google.common.io.Files; +import org.apache.flink.shaded.guava18.com.google.common.base.Charsets; +import org.apache.flink.shaded.guava18.com.google.common.io.Files; + import org.junit.After; import org.junit.Before; import org.junit.Rule; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index a186d56e9c9c7..03f82d81e0c24 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -18,9 +18,11 @@ package org.apache.flink.runtime.io.network.buffer; -import com.google.common.collect.Lists; import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.util.event.EventListener; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.After; import org.junit.AfterClass; import org.junit.Before; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java index 8f8da9340cf18..fa62593387122 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.io.network.partition; -import com.google.common.collect.Lists; import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; @@ -29,6 +28,9 @@ import org.apache.flink.runtime.io.network.util.TestInfiniteBufferProvider; import org.apache.flink.runtime.io.network.util.TestPooledBufferProvider; import org.apache.flink.runtime.io.network.util.TestSubpartitionConsumer; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.AfterClass; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index fe819a419b749..e685f17bb1230 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import com.google.common.collect.Lists; import org.apache.flink.api.common.JobID; import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.execution.CancelTaskException; @@ -42,6 +41,9 @@ import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; import org.apache.flink.runtime.taskmanager.TaskActions; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; 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 1d30a9a999582..4a32d7380b342 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 @@ -18,7 +18,6 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import com.google.common.collect.Lists; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.ConnectionManager; @@ -27,6 +26,9 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.util.TestBufferFactory; import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import scala.Tuple2; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java index 339b6f44ca74e..d7e96439eac74 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestPooledBufferProvider.java @@ -18,13 +18,14 @@ package org.apache.flink.runtime.io.network.util; -import com.google.common.collect.Queues; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.buffer.BufferRecycler; import org.apache.flink.runtime.util.event.EventListener; +import org.apache.flink.shaded.guava18.com.google.common.collect.Queues; + import java.io.IOException; import java.util.Queue; import java.util.concurrent.ArrayBlockingQueue; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/BlockingBackChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/BlockingBackChannelTest.java index c9015e9dacb49..f7b5c1dee5e22 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/BlockingBackChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/BlockingBackChannelTest.java @@ -22,10 +22,10 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.iterative.io.SerializedUpdateBuffer; -import com.google.common.collect.Lists; import org.junit.Test; import org.mockito.Mockito; +import java.util.ArrayList; import java.util.List; import java.util.Random; import java.util.concurrent.ArrayBlockingQueue; @@ -46,7 +46,7 @@ public class BlockingBackChannelTest { public void multiThreaded() throws InterruptedException { BlockingQueue dataChannel = new ArrayBlockingQueue(1); - List actionLog = Lists.newArrayList(); + List actionLog = new ArrayList<>(); SerializedUpdateBuffer buffer = Mockito.mock(SerializedUpdateBuffer.class); BlockingBackChannel channel = new BlockingBackChannel(buffer); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/BrokerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/BrokerTest.java index e12cb32c52ff9..e462e082551f6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/BrokerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/iterative/concurrent/BrokerTest.java @@ -20,9 +20,9 @@ import org.apache.flink.util.Preconditions; -import com.google.common.collect.Lists; import org.junit.Test; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Random; @@ -52,7 +52,7 @@ void mediate(int subtasks) throws InterruptedException, ExecutionException { final ExecutorService executorService = Executors.newFixedThreadPool(subtasks * 2); try { - List> tasks = Lists.newArrayList(); + List> tasks = new ArrayList<>(); Broker broker = new Broker(); for (int subtask = 0; subtask < subtasks; subtask++) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java index 82b510ca6d38f..838b12458de74 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerStartupTest.java @@ -33,9 +33,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.util.StartupUtils; import org.apache.flink.util.NetUtils; - import org.apache.flink.util.OperatingSystem; import org.apache.flink.util.TestLogger; + import org.junit.After; import org.junit.Before; import org.junit.Rule; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java index eb4d96fb29759..9c781ec101c18 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleOrUpdateConsumersTest.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.jobmanager.scheduler; -import com.google.common.collect.Lists; - import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -31,6 +29,8 @@ import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.types.IntValue; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java index b265eaef13eb3..ac57d8d77077b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/AbstractOuterJoinTaskTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.operators; -import com.google.common.base.Throwables; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -37,6 +36,9 @@ import org.apache.flink.runtime.operators.testutils.InfiniteIntTupleIterator; import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator; import org.apache.flink.util.Collector; + +import org.apache.flink.shaded.guava18.com.google.common.base.Throwables; + import org.junit.Assert; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java index 266723a778fe8..3b4c705fc8054 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/LeftOuterJoinTaskTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.operators; -import com.google.common.base.Throwables; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.runtime.RuntimePairComparatorFactory; @@ -28,6 +27,9 @@ import org.apache.flink.runtime.operators.testutils.ExpectedTestException; import org.apache.flink.runtime.operators.testutils.InfiniteIntTupleIterator; import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator; + +import org.apache.flink.shaded.guava18.com.google.common.base.Throwables; + import org.junit.Assert; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java index 4d410316d89a6..7f1f2b465160b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/RightOuterJoinTaskTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.operators; -import com.google.common.base.Throwables; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.runtime.RuntimePairComparatorFactory; @@ -28,6 +27,9 @@ import org.apache.flink.runtime.operators.testutils.ExpectedTestException; import org.apache.flink.runtime.operators.testutils.InfiniteIntTupleIterator; import org.apache.flink.runtime.operators.testutils.UniformIntTupleGenerator; + +import org.apache.flink.shaded.guava18.com.google.common.base.Throwables; + import org.junit.Assert; import org.junit.Test; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/InPlaceMutableHashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/InPlaceMutableHashTableTest.java index 4db5ef80d087f..beeccecd21c82 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/InPlaceMutableHashTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/InPlaceMutableHashTableTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.operators.hash; -import com.google.common.collect.Ordering; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.util.CopyingListCollector; import org.apache.flink.api.common.typeutils.SameTypePairComparator; @@ -37,6 +36,9 @@ import org.apache.flink.runtime.operators.testutils.types.*; import org.apache.flink.util.Collector; import org.apache.flink.util.MutableObjectIterator; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Ordering; + import org.junit.Test; import java.io.EOFException; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java index f08ad2d98be0b..6debff75a3e23 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java @@ -21,7 +21,6 @@ import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; -import com.google.common.base.Joiner; import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.FoldFunction; @@ -70,6 +69,9 @@ import org.apache.flink.util.IOUtils; import org.apache.flink.util.StateMigrationException; import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.base.Joiner; + import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala index ddbb82dc16630..02f83fd7fdd29 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingUtils.scala @@ -24,7 +24,6 @@ import java.util.concurrent._ import akka.actor.{ActorRef, ActorSystem, Kill, Props} import akka.pattern.{Patterns, ask} -import com.google.common.util.concurrent.MoreExecutors import com.typesafe.config.ConfigFactory import grizzled.slf4j.Logger import org.apache.flink.api.common.time.Time @@ -130,7 +129,8 @@ object TestingUtils { * * @return Direct [[ExecutionContext]] which executes runnables directly */ - def directExecutionContext = ExecutionContext.fromExecutor(MoreExecutors.directExecutor()) + def directExecutionContext = ExecutionContext + .fromExecutor(org.apache.flink.runtime.concurrent.Executors.directExecutor()) /** @return A new [[QueuedActionExecutionContext]] */ def queuedActionExecutionContext = { diff --git a/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml index b539f96a5018c..61897d1ca12fb 100644 --- a/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml +++ b/flink-shaded-curator/flink-shaded-curator-recipes/pom.xml @@ -41,13 +41,6 @@ under the License. curator-recipes ${curator.version} - - - - com.google.guava - guava - ${guava.version} - @@ -69,6 +62,16 @@ under the License. org.apache.curator:* + + + com.google + org.apache.flink.curator.shaded.com.google + + com.google.protobuf.** + com.google.inject.** + + + diff --git a/flink-shaded-curator/flink-shaded-curator-test/pom.xml b/flink-shaded-curator/flink-shaded-curator-test/pom.xml index 751b590efe0c8..2a181621e1628 100644 --- a/flink-shaded-curator/flink-shaded-curator-test/pom.xml +++ b/flink-shaded-curator/flink-shaded-curator-test/pom.xml @@ -69,6 +69,7 @@ org.apache.curator:curator-test + com.google.guava:guava @@ -76,6 +77,14 @@ org.apache.curator org.apache.flink.shaded.org.apache.curator + + com.google + org.apache.flink.curator.shaded.com.google + + com.google.protobuf.** + com.google.inject.** + + diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml index aefee5d3d425b..2683546274e2d 100644 --- a/flink-streaming-java/pom.xml +++ b/flink-streaming-java/pom.xml @@ -56,6 +56,11 @@ under the License. ${project.version} + + org.apache.flink + flink-shaded-guava + + org.apache.commons commons-math3 @@ -68,12 +73,6 @@ under the License. 2.0.6 - - com.google.guava - guava - ${guava.version} - - diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java index cfaa4b12185fd..dd6148c32f0a4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java @@ -27,9 +27,10 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; -import com.google.common.hash.HashFunction; -import com.google.common.hash.Hasher; -import com.google.common.hash.Hashing; +import org.apache.flink.shaded.guava18.com.google.common.hash.HashFunction; +import org.apache.flink.shaded.guava18.com.google.common.hash.Hasher; +import org.apache.flink.shaded.guava18.com.google.common.hash.Hashing; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java index 4be6b6e20d967..0beae32435d4c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/SplitStream.java @@ -22,7 +22,7 @@ import org.apache.flink.streaming.api.transformations.SelectTransformation; import org.apache.flink.streaming.api.transformations.SplitTransformation; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; /** * The SplitStream represents an operator that has been split using an diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java index bb9e47b6c6a70..9bbcec0c09198 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java @@ -25,9 +25,10 @@ import org.apache.flink.streaming.api.transformations.StreamTransformation; import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; -import com.google.common.hash.HashFunction; -import com.google.common.hash.Hasher; -import com.google.common.hash.Hashing; +import org.apache.flink.shaded.guava18.com.google.common.hash.HashFunction; +import org.apache.flink.shaded.guava18.com.google.common.hash.Hasher; +import org.apache.flink.shaded.guava18.com.google.common.hash.Hashing; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java index b36ad22a7ee07..28496fc31be23 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.operators.ChainingStrategy; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.Collections; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java index e5d7c3ac297c1..03a4e52955743 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.api.operators.ChainingStrategy; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java index bc1be5b8ed3d4..c9362866eed92 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java index 942d019a30984..6f30e0f6afa76 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java @@ -22,7 +22,7 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java index 6f47264cd5830..2f867cb72e187 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.api.operators.ChainingStrategy; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java index faa01f47cf188..faa033b106531 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SideOutputTransformation.java @@ -21,7 +21,7 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.util.OutputTag; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java index 5534cb994e8e7..30ef35eea6d8e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java @@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.StreamSink; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java index 148478a4e380f..d20276111c81e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java @@ -22,7 +22,7 @@ import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.operators.ChainingStrategy; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java index 7f561c51061ea..5ee055c97e3b2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java index bc522e791cc96..2bca7571e2833 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.api.operators.ChainingStrategy; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.Collection; import java.util.List; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java index 57fec10711b7c..5eeaff8dde4ab 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/evictors/DeltaEvictor.java @@ -23,7 +23,7 @@ import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.streaming.runtime.operators.windowing.TimestampedValue; -import com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; import java.util.Iterator; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java index d78de097441c5..29602af171b54 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperator.java @@ -38,9 +38,9 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.OutputTag; -import com.google.common.base.Function; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.base.Function; +import org.apache.flink.shaded.guava18.com.google.common.collect.FluentIterable; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; import java.util.Collection; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java index acb531dc8a284..db9622b4e48b9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java @@ -34,7 +34,8 @@ import org.apache.flink.streaming.util.MockContext; import org.apache.flink.streaming.util.keys.KeySelectorUtil; -import com.google.common.collect.ImmutableList; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableList; + import org.junit.Test; import java.io.Serializable; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ContinuousEventTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ContinuousEventTimeTriggerTest.java index 9c14a9fdd2598..f0af9c2b5bf09 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ContinuousEventTimeTriggerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ContinuousEventTimeTriggerTest.java @@ -25,7 +25,8 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import java.util.Collection; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CountTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CountTriggerTest.java index 38dd01d72057d..47fd9c228f5fc 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CountTriggerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/CountTriggerTest.java @@ -23,7 +23,8 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import static org.junit.Assert.assertEquals; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java index 23af8384b306f..5c4c989338f50 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeSessionWindowsTest.java @@ -28,7 +28,8 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import org.mockito.Matchers; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeTriggerTest.java index 2bcc19284c68f..f54367b699a2a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeTriggerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EventTimeTriggerTest.java @@ -23,7 +23,8 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import static org.junit.Assert.assertEquals; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java index 0c45d0318133f..019facabf5975 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/MergingWindowSetTest.java @@ -30,7 +30,8 @@ import org.apache.flink.streaming.api.windowing.triggers.Trigger; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import org.mockito.Matchers; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java index ceda3b90cde35..f49799cee99e3 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeSessionWindowsTest.java @@ -28,7 +28,8 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.util.TestLogger; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import org.mockito.Matchers; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeTriggerTest.java index 791eb424a6eda..7e78854583aed 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeTriggerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/ProcessingTimeTriggerTest.java @@ -23,7 +23,8 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + import org.junit.Test; import static org.junit.Assert.assertEquals; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index 42c6c6f9aeb78..acdf45a635b49 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -68,8 +68,9 @@ import org.apache.flink.util.OutputTag; import org.apache.flink.util.TestLogger; -import com.google.common.base.Joiner; -import com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.base.Joiner; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; + import org.junit.Assert; import org.junit.Test; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java index 64894484cb75b..807b68c80b574 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/TestHarnessUtil.java @@ -21,7 +21,8 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; + import org.junit.Assert; import java.util.ArrayList; diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index 85d90b3684edb..a0c68a81a75b0 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -193,6 +193,11 @@ under the License. test + + org.apache.flink + flink-shaded-guava + + org.scalatest scalatest_${scala.binary.version} diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index a3d45dd48f9ad..bf1bb1bb987a9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -74,11 +74,13 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; +import org.apache.flink.shaded.guava18.com.google.common.collect.HashMultimap; +import org.apache.flink.shaded.guava18.com.google.common.collect.Multimap; + import akka.actor.ActorRef; import akka.actor.ActorSystem; import akka.testkit.JavaTestKit; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; + import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java index f19d6904f03cd..530e97310e3f9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java @@ -33,7 +33,8 @@ import org.apache.flink.streaming.api.operators.StreamGroupedFold; import org.apache.flink.streaming.api.operators.StreamGroupedReduce; -import com.google.common.collect.EvictingQueue; +import org.apache.flink.shaded.guava18.com.google.common.collect.EvictingQueue; + import org.junit.Assert; import java.util.Collections; diff --git a/pom.xml b/pom.xml index 6ed08fdac8374..561ec791868dc 100644 --- a/pom.xml +++ b/pom.xml @@ -220,10 +220,15 @@ under the License. netty here. It will overwrite Hadoop's guava dependency (even though we handle it separatly in the flink-shaded-hadoop module). - We can use all guava versions everywhere by adding it directly as a dependency to each project. --> + + org.apache.flink + flink-shaded-guava + 18.0-1.0 + + com.google.code.findbugs @@ -1259,19 +1264,8 @@ under the License. shading, the root pom would have to be Scala suffixed and thereby all other modules. --> org.apache.flink:force-shading - com.google.guava:* - - - com.google - org.apache.flink.shaded.com.google - - com.google.protobuf.** - com.google.inject.** - - - diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml index 79102fac31790..514453ee5f403 100644 --- a/tools/maven/checkstyle.xml +++ b/tools/maven/checkstyle.xml @@ -211,7 +211,7 @@ This file is based on the checkstyle file of Apache Beam. - + diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index 8a80341289dca..b19435eff4cc4 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -27,4 +27,13 @@ under the License. + + + + + From 4a4db0a7f9ed2d8f1590589be3a4b43789208e6c Mon Sep 17 00:00:00 2001 From: Ajay Tripathy <4tripathy@gmail.com> Date: Mon, 14 Aug 2017 12:20:36 -0700 Subject: [PATCH 012/129] [hotfix] [docs][metrics] Small typo fix in the scope section --- docs/monitoring/metrics.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/monitoring/metrics.md b/docs/monitoring/metrics.md index b8f4acce4150d..b71f4cf3f07cd 100644 --- a/docs/monitoring/metrics.md +++ b/docs/monitoring/metrics.md @@ -229,7 +229,7 @@ public class MyMapper extends RichMapFunction { ## Scope Every metric is assigned an identifier under which it will be reported that is based on 3 components: the user-provided name when registering the metric, an optional user-defined scope and a system-provided scope. -For example, if `A.B` is the sytem scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`. +For example, if `A.B` is the system scope, `C.D` the user scope and `E` the name, then the identifier for the metric will be `A.B.C.D.E`. You can configure which delimiter to use for the identifier (default: `.`) by setting the `metrics.scope.delimiter` key in `conf/flink-conf.yaml`. From 61ed131735d64786709f4d73b31c934c4a51d3fa Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 14 Aug 2017 23:14:13 +0200 Subject: [PATCH 013/129] [FLINK-7445] [GitHub] Remove FLINK-1234 reference from PR template This closes #4542. --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 703e1ce9d5975..f28b5ede8a389 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -7,7 +7,7 @@ - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue. - - Name the pull request in the form "[FLINK-1234] [component] Title of the pull request", where *FLINK-1234* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component. + - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component. Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`. - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review. From 9eeb5b21eb9595a26bfb975f952082cf81e314a1 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 14 Aug 2017 14:12:51 +0200 Subject: [PATCH 014/129] [FLINK-7443] [metrics] MetricFetcher store and deserializer fields now final This closes #4539. --- .../flink/runtime/webmonitor/metrics/MetricFetcher.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcher.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcher.java index 3af9c5631b852..3fe4d128fb0be 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcher.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/MetricFetcher.java @@ -58,8 +58,8 @@ public class MetricFetcher { private final Executor executor; private final Time timeout; - private MetricStore metrics = new MetricStore(); - private MetricDumpDeserializer deserializer = new MetricDumpDeserializer(); + private final MetricStore metrics = new MetricStore(); + private final MetricDumpDeserializer deserializer = new MetricDumpDeserializer(); private long lastUpdateTime; From 90c2185c4d972e5225946eadfbf8e448cd4caf10 Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Tue, 15 Aug 2017 17:07:24 +0800 Subject: [PATCH 015/129] Fix bug when user set offset to latest, it cannot work userDefinedOffset.toLowerCase() == "lastest" does not work in JAVA world --- .../streaming/connectors/eventhubs/FlinkEventHubConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java index 109519aa079e7..02cde081fb6b7 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -82,7 +82,7 @@ public FlinkEventHubConsumer(Properties eventhubsProps, KeyedDeserializationSche this.deserializer = deserializer; String userDefinedOffset = eventhubsProps.getProperty("eventhubs.auto.offset"); - if (userDefinedOffset != null && userDefinedOffset.toLowerCase() == "lastest"){ + if (userDefinedOffset != null && userDefinedOffset.toLowerCase().compareTo("lastest") == 0){ this.defaultEventhubInitOffset = PartitionReceiver.END_OF_STREAM; } else { From 5f97ac9281a443cd72b374c293e525ee5e9b4149 Mon Sep 17 00:00:00 2001 From: yew1eb Date: Thu, 10 Aug 2017 23:28:00 +0800 Subject: [PATCH 016/129] [FLINK-7415] [cassandra] Add example instructions for creating keyspace This closes #4519. --- .../batch/connectors/cassandra/example/BatchExample.java | 5 +++-- .../cassandra/example/CassandraPojoSinkExample.java | 3 ++- .../cassandra/example/CassandraTupleSinkExample.java | 3 ++- .../example/CassandraTupleWriteAheadSinkExample.java | 3 ++- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java index af21f2d4a5eab..20a020802630a 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/batch/connectors/cassandra/example/BatchExample.java @@ -34,8 +34,9 @@ /** * This is an example showing the to use the Cassandra Input-/OutputFormats in the Batch API. * - *

The example assumes that a table exists in a local cassandra database, according to the following query: - * CREATE TABLE test.batches (number int, strings text, PRIMARY KEY(number, strings)); + *

The example assumes that a table exists in a local cassandra database, according to the following queries: + * CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’}; + * CREATE TABLE IF NOT EXISTS test.batches (number int, strings text, PRIMARY KEY(number, strings)); */ public class BatchExample { private static final String INSERT_QUERY = "INSERT INTO test.batches (number, strings) VALUES (?,?);"; diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java index a38b73b72c991..01cd6e8048f27 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraPojoSinkExample.java @@ -32,7 +32,8 @@ * *

Pojo's have to be annotated with datastax annotations to work with this sink. * - *

The example assumes that a table exists in a local cassandra database, according to the following query: + *

The example assumes that a table exists in a local cassandra database, according to the following queries: + * CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’}; * CREATE TABLE IF NOT EXISTS test.message(body txt PRIMARY KEY) */ public class CassandraPojoSinkExample { diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java index ce2326f4e569c..72013d5141a3f 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleSinkExample.java @@ -31,7 +31,8 @@ /** * This is an example showing the to use the Tuple Cassandra Sink in the Streaming API. * - *

The example assumes that a table exists in a local cassandra database, according to the following query: + *

The example assumes that a table exists in a local cassandra database, according to the following queries: + * CREATE KEYSPACE IF NOT EXISTS test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’}; * CREATE TABLE IF NOT EXISTS test.writetuple(element1 text PRIMARY KEY, element2 int) */ public class CassandraTupleSinkExample { diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java index 38618feaf7771..8cab311be7f90 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/example/CassandraTupleWriteAheadSinkExample.java @@ -36,7 +36,8 @@ /** * This is an example showing the to use the Cassandra Sink (with write-ahead log) in the Streaming API. * - *

The example assumes that a table exists in a local cassandra database, according to the following query: + *

The example assumes that a table exists in a local cassandra database, according to the following queries: + * CREATE KEYSPACE IF NOT EXISTS example WITH replication = {'class': 'SimpleStrategy', 'replication_factor': ‘1’}; * CREATE TABLE example.values (id text, count int, PRIMARY KEY(id)); * *

Important things to note are that checkpointing is enabled, a StateBackend is set and the enableWriteAheadLog() call From ff27e206b36a0cec4bb52b6a69ecd704aa237259 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 9 Aug 2017 16:26:08 +0200 Subject: [PATCH 017/129] [hotfix][misc] Fix logging of local port in NetworkFailuresProxy Previously if localPort was set 0, actually obtained/bind port was not logged anywhere. Now we print local port after binding. --- .../java/org/apache/flink/networking/NetworkFailuresProxy.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailuresProxy.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailuresProxy.java index 70300494a7e0e..5531811ca8c30 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailuresProxy.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailuresProxy.java @@ -53,8 +53,6 @@ public class NetworkFailuresProxy implements AutoCloseable { private final Set networkFailureHandlers = Collections.newSetFromMap(new ConcurrentHashMap<>()); public NetworkFailuresProxy(int localPort, String remoteHost, int remotePort) { - LOG.info("Proxying [*:{}] to [{}:{}]", localPort, remoteHost, remotePort); - // Configure the bootstrap. serverBootstrap = new ServerBootstrap( new NioServerSocketChannelFactory(executor, executor)); @@ -83,6 +81,7 @@ public ChannelPipeline getPipeline() throws Exception { }); channel = serverBootstrap.bind(new InetSocketAddress(localPort)); + LOG.info("Proxying [*:{}] to [{}:{}]", getLocalPort(), remoteHost, remotePort); } /** From be196f705a65bc6d42bba04ee2df2f48fc469db6 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 10 Aug 2017 10:29:17 +0200 Subject: [PATCH 018/129] [hotfix][misc] More verbose logging in NetworkFailuresProxy This closes #4515. --- .../apache/flink/networking/NetworkFailureHandler.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailureHandler.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailureHandler.java index 0ce0b12559058..07d23412d0952 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailureHandler.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/networking/NetworkFailureHandler.java @@ -92,7 +92,14 @@ public void channelOpen(ChannelHandlerContext context, ChannelStateEvent event) final Channel sourceChannel = event.getChannel(); sourceChannel.setReadable(false); - if (blocked.get()) { + boolean isBlocked = blocked.get(); + LOG.debug("Attempt to open proxy channel from [{}] to [{}:{}] in state [blocked = {}]", + sourceChannel.getLocalAddress(), + remoteHost, + remotePort, + isBlocked); + + if (isBlocked) { sourceChannel.close(); return; } From 3b0321aee4a409658399b53d8ff77b872e61e42c Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Wed, 9 Aug 2017 14:11:16 -0700 Subject: [PATCH 019/129] [FLINK-7405] [metrics] Reduce excessive warning logging from DatadogHttpReporter This closes #4508. --- .../org/apache/flink/metrics/datadog/DatadogHttpReporter.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java index a47b2bf13e94d..b7e1c24fa6034 100644 --- a/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java +++ b/flink-metrics/flink-metrics-datadog/src/main/java/org/apache/flink/metrics/datadog/DatadogHttpReporter.java @@ -31,6 +31,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.SocketTimeoutException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -140,6 +141,8 @@ public void report() { try { client.send(request); + } catch (SocketTimeoutException e) { + LOGGER.warn("Failed reporting metrics to Datadog because of socket timeout.", e.getMessage()); } catch (Exception e) { LOGGER.warn("Failed reporting metrics to Datadog.", e); } From b71154a734ea9f4489dffe1be6761efbb90cff41 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 26 Jun 2017 18:07:59 +0200 Subject: [PATCH 020/129] [FLINK-7213] Introduce state management by OperatorID in TaskManager --- .../state/RocksDBAsyncSnapshotTest.java | 21 +- .../checkpoint/CheckpointCoordinator.java | 7 +- .../CheckpointCoordinatorGateway.java | 2 +- .../runtime/checkpoint/OperatorState.java | 4 +- .../checkpoint/OperatorSubtaskState.java | 224 ++++++-- .../runtime/checkpoint/PendingCheckpoint.java | 54 +- .../RoundRobinOperatorStateRepartitioner.java | 4 + .../checkpoint/StateAssignmentOperation.java | 177 ++++--- .../runtime/checkpoint/TaskStateSnapshot.java | 139 +++++ .../savepoint/SavepointV2Serializer.java | 20 +- .../deployment/TaskDeploymentDescriptor.java | 8 +- .../flink/runtime/execution/Environment.java | 9 +- .../runtime/executiongraph/Execution.java | 8 +- .../executiongraph/ExecutionVertex.java | 8 +- .../runtime/jobgraph/tasks/StatefulTask.java | 8 +- .../flink/runtime/jobmaster/JobMaster.java | 5 +- .../checkpoint/AcknowledgeCheckpoint.java | 8 +- .../state/StateInitializationContextImpl.java | 11 +- .../flink/runtime/state/TaskStateHandles.java | 172 ------ .../rpc/RpcCheckpointResponder.java | 4 +- .../ActorGatewayCheckpointResponder.java | 4 +- .../taskmanager/CheckpointResponder.java | 4 +- .../taskmanager/RuntimeEnvironment.java | 4 +- .../flink/runtime/taskmanager/Task.java | 8 +- .../CheckpointCoordinatorFailureTest.java | 49 +- .../checkpoint/CheckpointCoordinatorTest.java | 498 +++++++++--------- .../CheckpointStateRestoreTest.java | 49 +- .../CompletedCheckpointStoreTest.java | 2 +- .../checkpoint/PendingCheckpointTest.java | 2 +- .../TaskDeploymentDescriptorTest.java | 4 +- .../ExecutionVertexLocalityTest.java | 10 +- .../jobmanager/JobManagerHARecoveryTest.java | 60 +-- .../messages/CheckpointMessagesTest.java | 23 +- .../operators/testutils/DummyEnvironment.java | 4 +- .../operators/testutils/MockEnvironment.java | 6 +- .../taskmanager/TaskAsyncCallTest.java | 6 +- .../runtime/taskmanager/TaskStopTest.java | 26 +- .../runtime/util/JvmExitOnFatalErrorTest.java | 7 +- .../streaming/api/graph/StreamConfig.java | 13 +- .../api/graph/StreamingJobGraphGenerator.java | 11 +- .../api/operators/AbstractStreamOperator.java | 19 +- .../api/operators/StreamOperator.java | 6 +- .../runtime/tasks/OperatorStateHandles.java | 19 - .../streaming/runtime/tasks/StreamTask.java | 196 +++---- ...bstractUdfStreamOperatorLifecycleTest.java | 5 +- .../async/AsyncWaitOperatorTest.java | 16 +- .../runtime/io/BarrierBufferTest.java | 4 +- .../runtime/io/BarrierTrackerTest.java | 4 +- .../operators/StreamTaskTimerTest.java | 2 + .../TestProcessingTimeServiceTest.java | 2 + .../tasks/BlockingCheckpointsTest.java | 2 + .../tasks/InterruptSensitiveRestoreTest.java | 55 +- .../runtime/tasks/OneInputStreamTaskTest.java | 34 +- .../SourceExternalCheckpointTriggerTest.java | 2 + .../runtime/tasks/SourceStreamTaskTest.java | 3 + .../runtime/tasks/StreamMockEnvironment.java | 4 +- .../StreamTaskCancellationBarrierTest.java | 3 + .../tasks/StreamTaskTerminationTest.java | 2 + .../runtime/tasks/StreamTaskTest.java | 79 ++- .../runtime/tasks/StreamTaskTestHarness.java | 2 + .../runtime/tasks/TwoInputStreamTaskTest.java | 5 + .../AbstractStreamOperatorTestHarness.java | 22 +- .../test/checkpointing/SavepointITCase.java | 2 +- 63 files changed, 1185 insertions(+), 986 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java diff --git a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java index d2edf0efaf768..c752e53ccd17c 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBAsyncSnapshotTest.java @@ -32,8 +32,10 @@ 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.SubtaskState; +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.OperatorID; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; @@ -74,6 +76,7 @@ import java.lang.reflect.Field; import java.net.URI; import java.util.Arrays; +import java.util.Map; import java.util.UUID; import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; @@ -81,7 +84,7 @@ import java.util.concurrent.RunnableFuture; import java.util.concurrent.TimeUnit; -import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyInt; @@ -137,6 +140,7 @@ public String getKey(String value) throws Exception { streamConfig.setStateBackend(backend); streamConfig.setStreamOperator(new AsyncCheckpointOperator()); + streamConfig.setOperatorID(new OperatorID()); final OneShotLatch delayCheckpointLatch = new OneShotLatch(); final OneShotLatch ensureCheckpointLatch = new OneShotLatch(); @@ -152,7 +156,7 @@ public String getKey(String value) throws Exception { public void acknowledgeCheckpoint( long checkpointId, CheckpointMetrics checkpointMetrics, - SubtaskState checkpointStateHandles) { + TaskStateSnapshot checkpointStateHandles) { super.acknowledgeCheckpoint(checkpointId, checkpointMetrics); @@ -164,8 +168,16 @@ public void acknowledgeCheckpoint( throw new RuntimeException(e); } + boolean hasManagedKeyedState = false; + for (Map.Entry entry : checkpointStateHandles.getSubtaskStateMappings()) { + OperatorSubtaskState state = entry.getValue(); + if (state != null) { + hasManagedKeyedState |= state.getManagedKeyedState() != null; + } + } + // should be one k/v state - assertNotNull(checkpointStateHandles.getManagedKeyedState()); + assertTrue(hasManagedKeyedState); // we now know that the checkpoint went through ensureCheckpointLatch.trigger(); @@ -241,6 +253,7 @@ public String getKey(String value) throws Exception { streamConfig.setStateBackend(backend); streamConfig.setStreamOperator(new AsyncCheckpointOperator()); + streamConfig.setOperatorID(new OperatorID()); StreamMockEnvironment mockEnv = new StreamMockEnvironment( testHarness.jobConfig, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 6f4186745e95d..0b64a737fef7b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -40,7 +40,6 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.state.SharedStateRegistry; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; @@ -1016,7 +1015,7 @@ int getNumScheduledTasks() { * Restores the latest checkpointed state. * * @param tasks Map of job vertices to restore. State for these vertices is - * restored via {@link Execution#setInitialState(TaskStateHandles)}. + * restored via {@link Execution#setInitialState(TaskStateSnapshot)}. * @param errorIfNoCheckpoint Fail if no completed checkpoint is available to * restore from. * @param allowNonRestoredState Allow checkpoint state that cannot be mapped @@ -1102,7 +1101,7 @@ public boolean restoreLatestCheckpointedState( * mapped to any job vertex in tasks. * @param tasks Map of job vertices to restore. State for these * vertices is restored via - * {@link Execution#setInitialState(TaskStateHandles)}. + * {@link Execution#setInitialState(TaskStateSnapshot)}. * @param userClassLoader The class loader to resolve serialized classes in * legacy savepoint versions. */ @@ -1256,7 +1255,7 @@ private void discardSubtaskState( final JobID jobId, final ExecutionAttemptID executionAttemptID, final long checkpointId, - final SubtaskState subtaskState) { + final TaskStateSnapshot subtaskState) { if (subtaskState != null) { executor.execute(new Runnable() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java index 43d66ee719604..22244f6cb8d51 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorGateway.java @@ -29,7 +29,7 @@ void acknowledgeCheckpoint( final ExecutionAttemptID executionAttemptID, final long checkpointId, final CheckpointMetrics checkpointMetrics, - final SubtaskState subtaskState); + final TaskStateSnapshot subtaskState); void declineCheckpoint( JobID jobID, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java index b15302835bb38..145ff6a978931 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorState.java @@ -30,8 +30,8 @@ import java.util.Objects; /** - * Simple container class which contains the raw/managed/legacy operator state and key-group state handles for the sub - * tasks of an operator. + * Simple container class which contains the raw/managed/legacy operator state and key-group state handles from all sub + * tasks of an operator and therefore represents the complete state of a logical operator. */ public class OperatorState implements CompositeStateHandle { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java index e2ae632a26b1b..296b5ab29dd32 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/OperatorSubtaskState.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.runtime.state.CompositeStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; @@ -25,13 +26,35 @@ import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; /** - * Container for the state of one parallel subtask of an operator. This is part of the {@link OperatorState}. + * This class encapsulates the state for one parallel instance of an operator. The complete state of a (logical) + * operator (e.g. a flatmap operator) consists of the union of all {@link OperatorSubtaskState}s from all + * parallel tasks that physically execute parallelized, physical instances of the operator. + * + *

The full state of the logical operator is represented by {@link OperatorState} which consists of + * {@link OperatorSubtaskState}s. + * + *

Typically, we expect all collections in this class to be of size 0 or 1, because there is up to one state handle + * produced per state type (e.g. managed-keyed, raw-operator, ...). In particular, this holds when taking a snapshot. + * The purpose of having the state handles in collections is that this class is also reused in restoring state. + * Under normal circumstances, the expected size of each collection is still 0 or 1, except for scale-down. In + * scale-down, one operator subtask can become responsible for the state of multiple previous subtasks. The collections + * can then store all the state handles that are relevant to build up the new subtask state. + * + *

There is no collection for legacy state because it is not rescalable. */ public class OperatorSubtaskState implements CompositeStateHandle { @@ -46,27 +69,32 @@ public class OperatorSubtaskState implements CompositeStateHandle { * Can be removed when we remove the APIs for non-repartitionable operator state. */ @Deprecated + @Nullable private final StreamStateHandle legacyOperatorState; /** * Snapshot from the {@link org.apache.flink.runtime.state.OperatorStateBackend}. */ - private final OperatorStateHandle managedOperatorState; + @Nonnull + private final Collection managedOperatorState; /** * Snapshot written using {@link org.apache.flink.runtime.state.OperatorStateCheckpointOutputStream}. */ - private final OperatorStateHandle rawOperatorState; + @Nonnull + private final Collection rawOperatorState; /** * Snapshot from {@link org.apache.flink.runtime.state.KeyedStateBackend}. */ - private final KeyedStateHandle managedKeyedState; + @Nonnull + private final Collection managedKeyedState; /** * Snapshot written using {@link org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream}. */ - private final KeyedStateHandle rawKeyedState; + @Nonnull + private final Collection rawKeyedState; /** * The state size. This is also part of the deserialized state handle. @@ -75,31 +103,79 @@ public class OperatorSubtaskState implements CompositeStateHandle { */ private final long stateSize; + @VisibleForTesting + public OperatorSubtaskState(StreamStateHandle legacyOperatorState) { + + this(legacyOperatorState, + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList()); + } + + /** + * Empty state. + */ + public OperatorSubtaskState() { + this(null); + } + public OperatorSubtaskState( StreamStateHandle legacyOperatorState, - OperatorStateHandle managedOperatorState, - OperatorStateHandle rawOperatorState, - KeyedStateHandle managedKeyedState, - KeyedStateHandle rawKeyedState) { + Collection managedOperatorState, + Collection rawOperatorState, + Collection managedKeyedState, + Collection rawKeyedState) { this.legacyOperatorState = legacyOperatorState; - this.managedOperatorState = managedOperatorState; - this.rawOperatorState = rawOperatorState; - this.managedKeyedState = managedKeyedState; - this.rawKeyedState = rawKeyedState; + this.managedOperatorState = Preconditions.checkNotNull(managedOperatorState); + this.rawOperatorState = Preconditions.checkNotNull(rawOperatorState); + this.managedKeyedState = Preconditions.checkNotNull(managedKeyedState); + this.rawKeyedState = Preconditions.checkNotNull(rawKeyedState); try { long calculateStateSize = getSizeNullSafe(legacyOperatorState); - calculateStateSize += getSizeNullSafe(managedOperatorState); - calculateStateSize += getSizeNullSafe(rawOperatorState); - calculateStateSize += getSizeNullSafe(managedKeyedState); - calculateStateSize += getSizeNullSafe(rawKeyedState); + calculateStateSize += sumAllSizes(managedOperatorState); + calculateStateSize += sumAllSizes(rawOperatorState); + calculateStateSize += sumAllSizes(managedKeyedState); + calculateStateSize += sumAllSizes(rawKeyedState); stateSize = calculateStateSize; } catch (Exception e) { throw new RuntimeException("Failed to get state size.", e); } } + /** + * For convenience because the size of the collections is typically 0 or 1. Null values are translated into empty + * Collections (except for legacy state). + */ + public OperatorSubtaskState( + StreamStateHandle legacyOperatorState, + OperatorStateHandle managedOperatorState, + OperatorStateHandle rawOperatorState, + KeyedStateHandle managedKeyedState, + KeyedStateHandle rawKeyedState) { + + this(legacyOperatorState, + singletonOrEmptyOnNull(managedOperatorState), + singletonOrEmptyOnNull(rawOperatorState), + singletonOrEmptyOnNull(managedKeyedState), + singletonOrEmptyOnNull(rawKeyedState)); + } + + private static Collection singletonOrEmptyOnNull(T element) { + return element != null ? Collections.singletonList(element) : Collections.emptyList(); + } + + private static long sumAllSizes(Collection stateObject) throws Exception { + long size = 0L; + for (StateObject object : stateObject) { + size += getSizeNullSafe(object); + } + + return size; + } + private static long getSizeNullSafe(StateObject stateObject) throws Exception { return stateObject != null ? stateObject.getStateSize() : 0L; } @@ -111,36 +187,58 @@ private static long getSizeNullSafe(StateObject stateObject) throws Exception { * Can be removed when we remove the APIs for non-repartitionable operator state. */ @Deprecated + @Nullable public StreamStateHandle getLegacyOperatorState() { return legacyOperatorState; } - public OperatorStateHandle getManagedOperatorState() { + /** + * Returns a handle to the managed operator state. + */ + @Nonnull + public Collection getManagedOperatorState() { return managedOperatorState; } - public OperatorStateHandle getRawOperatorState() { + /** + * Returns a handle to the raw operator state. + */ + @Nonnull + public Collection getRawOperatorState() { return rawOperatorState; } - public KeyedStateHandle getManagedKeyedState() { + /** + * Returns a handle to the managed keyed state. + */ + @Nonnull + public Collection getManagedKeyedState() { return managedKeyedState; } - public KeyedStateHandle getRawKeyedState() { + /** + * Returns a handle to the raw keyed state. + */ + @Nonnull + public Collection getRawKeyedState() { return rawKeyedState; } @Override public void discardState() { try { - StateUtil.bestEffortDiscardAllStateObjects( - Arrays.asList( - legacyOperatorState, - managedOperatorState, - rawOperatorState, - managedKeyedState, - rawKeyedState)); + List toDispose = + new ArrayList<>(1 + + managedOperatorState.size() + + rawOperatorState.size() + + managedKeyedState.size() + + rawKeyedState.size()); + toDispose.add(legacyOperatorState); + toDispose.addAll(managedOperatorState); + toDispose.addAll(rawOperatorState); + toDispose.addAll(managedKeyedState); + toDispose.addAll(rawKeyedState); + StateUtil.bestEffortDiscardAllStateObjects(toDispose); } catch (Exception e) { LOG.warn("Error while discarding operator states.", e); } @@ -148,12 +246,17 @@ public void discardState() { @Override public void registerSharedStates(SharedStateRegistry sharedStateRegistry) { - if (managedKeyedState != null) { - managedKeyedState.registerSharedStates(sharedStateRegistry); - } + registerSharedState(sharedStateRegistry, managedKeyedState); + registerSharedState(sharedStateRegistry, rawKeyedState); + } - if (rawKeyedState != null) { - rawKeyedState.registerSharedStates(sharedStateRegistry); + private static void registerSharedState( + SharedStateRegistry sharedStateRegistry, + Iterable stateHandles) { + for (KeyedStateHandle stateHandle : stateHandles) { + if (stateHandle != null) { + stateHandle.registerSharedStates(sharedStateRegistry); + } } } @@ -175,44 +278,32 @@ public boolean equals(Object o) { OperatorSubtaskState that = (OperatorSubtaskState) o; - if (stateSize != that.stateSize) { + if (getStateSize() != that.getStateSize()) { return false; } - - if (legacyOperatorState != null ? - !legacyOperatorState.equals(that.legacyOperatorState) - : that.legacyOperatorState != null) { + if (getLegacyOperatorState() != null ? !getLegacyOperatorState().equals(that.getLegacyOperatorState()) : that.getLegacyOperatorState() != null) { return false; } - if (managedOperatorState != null ? - !managedOperatorState.equals(that.managedOperatorState) - : that.managedOperatorState != null) { + if (!getManagedOperatorState().equals(that.getManagedOperatorState())) { return false; } - if (rawOperatorState != null ? - !rawOperatorState.equals(that.rawOperatorState) - : that.rawOperatorState != null) { + if (!getRawOperatorState().equals(that.getRawOperatorState())) { return false; } - if (managedKeyedState != null ? - !managedKeyedState.equals(that.managedKeyedState) - : that.managedKeyedState != null) { + if (!getManagedKeyedState().equals(that.getManagedKeyedState())) { return false; } - return rawKeyedState != null ? - rawKeyedState.equals(that.rawKeyedState) - : that.rawKeyedState == null; - + return getRawKeyedState().equals(that.getRawKeyedState()); } @Override public int hashCode() { - int result = legacyOperatorState != null ? legacyOperatorState.hashCode() : 0; - result = 31 * result + (managedOperatorState != null ? managedOperatorState.hashCode() : 0); - result = 31 * result + (rawOperatorState != null ? rawOperatorState.hashCode() : 0); - result = 31 * result + (managedKeyedState != null ? managedKeyedState.hashCode() : 0); - result = 31 * result + (rawKeyedState != null ? rawKeyedState.hashCode() : 0); - result = 31 * result + (int) (stateSize ^ (stateSize >>> 32)); + int result = getLegacyOperatorState() != null ? getLegacyOperatorState().hashCode() : 0; + result = 31 * result + getManagedOperatorState().hashCode(); + result = 31 * result + getRawOperatorState().hashCode(); + result = 31 * result + getManagedKeyedState().hashCode(); + result = 31 * result + getRawKeyedState().hashCode(); + result = 31 * result + (int) (getStateSize() ^ (getStateSize() >>> 32)); return result; } @@ -227,4 +318,21 @@ public String toString() { ", stateSize=" + stateSize + '}'; } + + public boolean hasState() { + return legacyOperatorState != null + || hasState(managedOperatorState) + || hasState(rawOperatorState) + || hasState(managedKeyedState) + || hasState(rawKeyedState); + } + + private boolean hasState(Iterable states) { + for (StateObject state : states) { + if (state != null) { + return true; + } + } + return false; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index 3472fc20abfd8..16231dd0a9663 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -25,19 +25,18 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.state.ChainedStateHandle; -import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.filesystem.FileStateHandle; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; + import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -353,13 +352,13 @@ private CompletedCheckpoint finalizeInternal( * Acknowledges the task with the given execution attempt id and the given subtask state. * * @param executionAttemptId of the acknowledged task - * @param subtaskState of the acknowledged task + * @param operatorSubtaskStates of the acknowledged task * @param metrics Checkpoint metrics for the stats * @return TaskAcknowledgeResult of the operation */ public TaskAcknowledgeResult acknowledgeTask( ExecutionAttemptID executionAttemptId, - SubtaskState subtaskState, + TaskStateSnapshot operatorSubtaskStates, CheckpointMetrics metrics) { synchronized (lock) { @@ -383,21 +382,19 @@ public TaskAcknowledgeResult acknowledgeTask( int subtaskIndex = vertex.getParallelSubtaskIndex(); long ackTimestamp = System.currentTimeMillis(); - long stateSize = 0; - if (subtaskState != null) { - stateSize = subtaskState.getStateSize(); - - @SuppressWarnings("deprecation") - ChainedStateHandle nonPartitionedState = - subtaskState.getLegacyOperatorState(); - ChainedStateHandle partitioneableState = - subtaskState.getManagedOperatorState(); - ChainedStateHandle rawOperatorState = - subtaskState.getRawOperatorState(); - - // break task state apart into separate operator states - for (int x = 0; x < operatorIDs.size(); x++) { - OperatorID operatorID = operatorIDs.get(x); + long stateSize = 0L; + + if (operatorSubtaskStates != null) { + for (OperatorID operatorID : operatorIDs) { + + OperatorSubtaskState operatorSubtaskState = + operatorSubtaskStates.getSubtaskStateByOperatorID(operatorID); + + // if no real operatorSubtaskState was reported, we insert an empty state + if (operatorSubtaskState == null) { + operatorSubtaskState = new OperatorSubtaskState(); + } + OperatorState operatorState = operatorStates.get(operatorID); if (operatorState == null) { @@ -408,23 +405,8 @@ public TaskAcknowledgeResult acknowledgeTask( operatorStates.put(operatorID, operatorState); } - KeyedStateHandle managedKeyedState = null; - KeyedStateHandle rawKeyedState = null; - - // only the head operator retains the keyed state - if (x == operatorIDs.size() - 1) { - managedKeyedState = subtaskState.getManagedKeyedState(); - rawKeyedState = subtaskState.getRawKeyedState(); - } - - OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( - nonPartitionedState != null ? nonPartitionedState.get(x) : null, - partitioneableState != null ? partitioneableState.get(x) : null, - rawOperatorState != null ? rawOperatorState.get(x) : null, - managedKeyedState, - rawKeyedState); - operatorState.putState(subtaskIndex, operatorSubtaskState); + stateSize += operatorSubtaskState.getStateSize(); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java index 046096fc85c08..4513ef80b32b1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/RoundRobinOperatorStateRepartitioner.java @@ -89,6 +89,10 @@ private GroupByStateNameResults groupByStateName( for (OperatorStateHandle psh : previousParallelSubtaskStates) { + if (psh == null) { + continue; + } + for (Map.Entry e : psh.getStateNameToPartitionOffsets().entrySet()) { OperatorStateHandle.StateMetaInfo metaInfo = e.getValue(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java index 5712ea1d43827..b69285ed5a69e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperation.java @@ -23,15 +23,14 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -185,7 +184,8 @@ private void assignAttemptState(ExecutionJobVertex executionJobVertex, List(subNonPartitionableState), - subManagedOperatorState, - subRawOperatorState, - subKeyedState != null ? subKeyedState.f0 : null, - subKeyedState != null ? subKeyedState.f1 : null); + for (int i = 0; i < operatorIDs.size(); ++i) { + + OperatorID operatorID = operatorIDs.get(i); + + Collection rawKeyed = Collections.emptyList(); + Collection managedKeyed = Collections.emptyList(); + + // keyed state case + if (subKeyedState != null) { + managedKeyed = subKeyedState.f0; + rawKeyed = subKeyedState.f1; + } + + OperatorSubtaskState operatorSubtaskState = + new OperatorSubtaskState( + subNonPartitionableState.get(i), + subManagedOperatorState.get(i), + subRawOperatorState.get(i), + managedKeyed, + rawKeyed + ); + + taskState.putSubtaskStateByOperatorID(operatorID, operatorSubtaskState); + } - currentExecutionAttempt.setInitialState(taskStateHandles); + currentExecutionAttempt.setInitialState(taskState); } } } + private static boolean isHeadOperator(int opIdx, List operatorIDs) { + return opIdx == operatorIDs.size() - 1; + } public void checkParallelismPreconditions(List operatorStates, ExecutionJobVertex executionJobVertex) { @@ -239,18 +260,18 @@ private void reAssignSubPartitionableState( List> subManagedOperatorState, List> subRawOperatorState) { - if (newMangedOperatorStates.get(operatorIndex) != null) { - subManagedOperatorState.add(newMangedOperatorStates.get(operatorIndex).get(subTaskIndex)); + if (newMangedOperatorStates.get(operatorIndex) != null && !newMangedOperatorStates.get(operatorIndex).isEmpty()) { + Collection operatorStateHandles = newMangedOperatorStates.get(operatorIndex).get(subTaskIndex); + subManagedOperatorState.add(operatorStateHandles != null ? operatorStateHandles : Collections.emptyList()); } else { - subManagedOperatorState.add(null); + subManagedOperatorState.add(Collections.emptyList()); } - if (newRawOperatorStates.get(operatorIndex) != null) { - subRawOperatorState.add(newRawOperatorStates.get(operatorIndex).get(subTaskIndex)); + if (newRawOperatorStates.get(operatorIndex) != null && !newRawOperatorStates.get(operatorIndex).isEmpty()) { + Collection operatorStateHandles = newRawOperatorStates.get(operatorIndex).get(subTaskIndex); + subRawOperatorState.add(operatorStateHandles != null ? operatorStateHandles : Collections.emptyList()); } else { - subRawOperatorState.add(null); + subRawOperatorState.add(Collections.emptyList()); } - - } private Tuple2, Collection> reAssignSubKeyedStates( @@ -265,24 +286,22 @@ private Tuple2, Collection> reAss if (newParallelism == oldParallelism) { if (operatorState.getState(subTaskIndex) != null) { - KeyedStateHandle oldSubManagedKeyedState = operatorState.getState(subTaskIndex).getManagedKeyedState(); - KeyedStateHandle oldSubRawKeyedState = operatorState.getState(subTaskIndex).getRawKeyedState(); - subManagedKeyedState = oldSubManagedKeyedState != null ? Collections.singletonList( - oldSubManagedKeyedState) : null; - subRawKeyedState = oldSubRawKeyedState != null ? Collections.singletonList( - oldSubRawKeyedState) : null; + subManagedKeyedState = operatorState.getState(subTaskIndex).getManagedKeyedState(); + subRawKeyedState = operatorState.getState(subTaskIndex).getRawKeyedState(); } else { - subManagedKeyedState = null; - subRawKeyedState = null; + subManagedKeyedState = Collections.emptyList(); + subRawKeyedState = Collections.emptyList(); } } else { subManagedKeyedState = getManagedKeyedStateHandles(operatorState, keyGroupPartitions.get(subTaskIndex)); subRawKeyedState = getRawKeyedStateHandles(operatorState, keyGroupPartitions.get(subTaskIndex)); } - if (subManagedKeyedState == null && subRawKeyedState == null) { + + if (subManagedKeyedState.isEmpty() && subRawKeyedState.isEmpty()) { return null; + } else { + return new Tuple2<>(subManagedKeyedState, subRawKeyedState); } - return new Tuple2<>(subManagedKeyedState, subRawKeyedState); } @@ -318,7 +337,7 @@ private void reDistributePartitionableStates( List>> newManagedOperatorStates, List>> newRawOperatorStates) { - //collect the old partitionalbe state + //collect the old partitionable state List> oldManagedOperatorStates = new ArrayList<>(); List> oldRawOperatorStates = new ArrayList<>(); @@ -351,19 +370,16 @@ private void collectPartionableStates( for (int i = 0; i < operatorState.getParallelism(); i++) { OperatorSubtaskState operatorSubtaskState = operatorState.getState(i); if (operatorSubtaskState != null) { - if (operatorSubtaskState.getManagedOperatorState() != null) { - if (managedOperatorState == null) { - managedOperatorState = new ArrayList<>(); - } - managedOperatorState.add(operatorSubtaskState.getManagedOperatorState()); + + if (managedOperatorState == null) { + managedOperatorState = new ArrayList<>(); } + managedOperatorState.addAll(operatorSubtaskState.getManagedOperatorState()); - if (operatorSubtaskState.getRawOperatorState() != null) { - if (rawOperatorState == null) { - rawOperatorState = new ArrayList<>(); - } - rawOperatorState.add(operatorSubtaskState.getRawOperatorState()); + if (rawOperatorState == null) { + rawOperatorState = new ArrayList<>(); } + rawOperatorState.addAll(operatorSubtaskState.getRawOperatorState()); } } @@ -382,21 +398,19 @@ private void collectPartionableStates( * @return all managedKeyedStateHandles which have intersection with given KeyGroupRange */ public static List getManagedKeyedStateHandles( - OperatorState operatorState, - KeyGroupRange subtaskKeyGroupRange) { + OperatorState operatorState, + KeyGroupRange subtaskKeyGroupRange) { - List subtaskKeyedStateHandles = null; + List subtaskKeyedStateHandles = new ArrayList<>(); for (int i = 0; i < operatorState.getParallelism(); i++) { - if (operatorState.getState(i) != null && operatorState.getState(i).getManagedKeyedState() != null) { - KeyedStateHandle intersectedKeyedStateHandle = operatorState.getState(i).getManagedKeyedState().getIntersection(subtaskKeyGroupRange); + if (operatorState.getState(i) != null) { - if (intersectedKeyedStateHandle != null) { - if (subtaskKeyedStateHandles == null) { - subtaskKeyedStateHandles = new ArrayList<>(); - } - subtaskKeyedStateHandles.add(intersectedKeyedStateHandle); - } + Collection keyedStateHandles = operatorState.getState(i).getManagedKeyedState(); + extractIntersectingState( + keyedStateHandles, + subtaskKeyGroupRange, + subtaskKeyedStateHandles); } } @@ -415,22 +429,40 @@ public static List getRawKeyedStateHandles( OperatorState operatorState, KeyGroupRange subtaskKeyGroupRange) { - List subtaskKeyedStateHandles = null; + List extractedKeyedStateHandles = new ArrayList<>(); for (int i = 0; i < operatorState.getParallelism(); i++) { - if (operatorState.getState(i) != null && operatorState.getState(i).getRawKeyedState() != null) { - KeyedStateHandle intersectedKeyedStateHandle = operatorState.getState(i).getRawKeyedState().getIntersection(subtaskKeyGroupRange); + if (operatorState.getState(i) != null) { + Collection rawKeyedState = operatorState.getState(i).getRawKeyedState(); + extractIntersectingState( + rawKeyedState, + subtaskKeyGroupRange, + extractedKeyedStateHandles); + } + } + + return extractedKeyedStateHandles; + } + + /** + * Extracts certain key group ranges from the given state handles and adds them to the collector. + */ + private static void extractIntersectingState( + Collection originalSubtaskStateHandles, + KeyGroupRange rangeToExtract, + List extractedStateCollector) { + + for (KeyedStateHandle keyedStateHandle : originalSubtaskStateHandles) { + + if (keyedStateHandle != null) { + + KeyedStateHandle intersectedKeyedStateHandle = keyedStateHandle.getIntersection(rangeToExtract); if (intersectedKeyedStateHandle != null) { - if (subtaskKeyedStateHandles == null) { - subtaskKeyedStateHandles = new ArrayList<>(); - } - subtaskKeyedStateHandles.add(intersectedKeyedStateHandle); + extractedStateCollector.add(intersectedKeyedStateHandle); } } } - - return subtaskKeyedStateHandles; } /** @@ -554,7 +586,7 @@ public static List> applyRepartitioner( int newParallelism) { if (chainOpParallelStates == null) { - return null; + return Collections.emptyList(); } //We only redistribute if the parallelism of the operator changed from previous executions @@ -567,20 +599,23 @@ public static List> applyRepartitioner( List> repackStream = new ArrayList<>(newParallelism); for (OperatorStateHandle operatorStateHandle : chainOpParallelStates) { - Map partitionOffsets = + if (operatorStateHandle != null) { + Map partitionOffsets = operatorStateHandle.getStateNameToPartitionOffsets(); - for (OperatorStateHandle.StateMetaInfo metaInfo : partitionOffsets.values()) { - // if we find any broadcast state, we cannot take the shortcut and need to go through repartitioning - if (OperatorStateHandle.Mode.BROADCAST.equals(metaInfo.getDistributionMode())) { - return opStateRepartitioner.repartitionState( + for (OperatorStateHandle.StateMetaInfo metaInfo : partitionOffsets.values()) { + + // if we find any broadcast state, we cannot take the shortcut and need to go through repartitioning + if (OperatorStateHandle.Mode.BROADCAST.equals(metaInfo.getDistributionMode())) { + return opStateRepartitioner.repartitionState( chainOpParallelStates, newParallelism); + } } - } - repackStream.add(Collections.singletonList(operatorStateHandle)); + repackStream.add(Collections.singletonList(operatorStateHandle)); + } } return repackStream; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java new file mode 100644 index 0000000000000..c416f3f641c10 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java @@ -0,0 +1,139 @@ +/* + * 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.checkpoint; + +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.CompositeStateHandle; +import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.StateUtil; +import org.apache.flink.util.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * This class encapsulates state handles to the snapshots of all operator instances executed within one task. A task + * can run multiple operator instances as a result of operator chaining, and all operator instances from the chain can + * register their state under their operator id. Each operator instance is a physical execution responsible for + * processing a partition of the data that goes through a logical operator. This partitioning happens to parallelize + * execution of logical operators, e.g. distributing a map function. + * + *

One instance of this class contains the information that one task will send to acknowledge a checkpoint request by + * the checkpoint coordinator. Tasks run operator instances in parallel, so the union of all + * {@link TaskStateSnapshot} that are collected by the checkpoint coordinator from all tasks represent the whole + * state of a job at the time of the checkpoint. + * + *

This class should be called TaskState once the old class with this name that we keep for backwards + * compatibility goes away. + */ +public class TaskStateSnapshot implements CompositeStateHandle { + + private static final long serialVersionUID = 1L; + + /** Mapping from an operator id to the state of one subtask of this operator */ + private final Map subtaskStatesByOperatorID; + + public TaskStateSnapshot() { + this(10); + } + + public TaskStateSnapshot(int size) { + this(new HashMap(size)); + } + + public TaskStateSnapshot(Map subtaskStatesByOperatorID) { + this.subtaskStatesByOperatorID = Preconditions.checkNotNull(subtaskStatesByOperatorID); + } + + /** + * Returns the subtask state for the given operator id (or null if not contained). + */ + public OperatorSubtaskState getSubtaskStateByOperatorID(OperatorID operatorID) { + return subtaskStatesByOperatorID.get(operatorID); + } + + /** + * Maps the given operator id to the given subtask state. Returns the subtask state of a previous mapping, if such + * a mapping existed or null otherwise. + */ + public OperatorSubtaskState putSubtaskStateByOperatorID(OperatorID operatorID, OperatorSubtaskState state) { + return subtaskStatesByOperatorID.put(operatorID, Preconditions.checkNotNull(state)); + } + + /** + * Returns the set of all mappings from operator id to the corresponding subtask state. + */ + public Set> getSubtaskStateMappings() { + return subtaskStatesByOperatorID.entrySet(); + } + + @Override + public void discardState() throws Exception { + StateUtil.bestEffortDiscardAllStateObjects(subtaskStatesByOperatorID.values()); + } + + @Override + public long getStateSize() { + long size = 0L; + + for (OperatorSubtaskState subtaskState : subtaskStatesByOperatorID.values()) { + if (subtaskState != null) { + size += subtaskState.getStateSize(); + } + } + + return size; + } + + @Override + public void registerSharedStates(SharedStateRegistry stateRegistry) { + for (OperatorSubtaskState operatorSubtaskState : subtaskStatesByOperatorID.values()) { + if (operatorSubtaskState != null) { + operatorSubtaskState.registerSharedStates(stateRegistry); + } + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + TaskStateSnapshot that = (TaskStateSnapshot) o; + + return subtaskStatesByOperatorID.equals(that.subtaskStatesByOperatorID); + } + + @Override + public int hashCode() { + return subtaskStatesByOperatorID.hashCode(); + } + + @Override + public String toString() { + return "TaskOperatorSubtaskStates{" + + "subtaskStatesByOperatorID=" + subtaskStatesByOperatorID + + '}'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java index 4cbbfcfba8b5b..15628a0429e31 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/savepoint/SavepointV2Serializer.java @@ -240,6 +240,18 @@ private MasterState deserializeMasterState(DataInputStream dis) throws IOExcepti // task state (de)serialization methods // ------------------------------------------------------------------------ + private static T extractSingleton(Collection collection) { + if (collection == null || collection.isEmpty()) { + return null; + } + + if (collection.size() == 1) { + return collection.iterator().next(); + } else { + throw new IllegalStateException("Expected singleton collection, but found size: " + collection.size()); + } + } + private static void serializeSubtaskState(OperatorSubtaskState subtaskState, DataOutputStream dos) throws IOException { dos.writeLong(-1); @@ -252,7 +264,7 @@ private static void serializeSubtaskState(OperatorSubtaskState subtaskState, Dat serializeStreamStateHandle(nonPartitionableState, dos); } - OperatorStateHandle operatorStateBackend = subtaskState.getManagedOperatorState(); + OperatorStateHandle operatorStateBackend = extractSingleton(subtaskState.getManagedOperatorState()); len = operatorStateBackend != null ? 1 : 0; dos.writeInt(len); @@ -260,7 +272,7 @@ private static void serializeSubtaskState(OperatorSubtaskState subtaskState, Dat serializeOperatorStateHandle(operatorStateBackend, dos); } - OperatorStateHandle operatorStateFromStream = subtaskState.getRawOperatorState(); + OperatorStateHandle operatorStateFromStream = extractSingleton(subtaskState.getRawOperatorState()); len = operatorStateFromStream != null ? 1 : 0; dos.writeInt(len); @@ -268,10 +280,10 @@ private static void serializeSubtaskState(OperatorSubtaskState subtaskState, Dat serializeOperatorStateHandle(operatorStateFromStream, dos); } - KeyedStateHandle keyedStateBackend = subtaskState.getManagedKeyedState(); + KeyedStateHandle keyedStateBackend = extractSingleton(subtaskState.getManagedKeyedState()); serializeKeyedStateHandle(keyedStateBackend, dos); - KeyedStateHandle keyedStateStream = subtaskState.getRawKeyedState(); + KeyedStateHandle keyedStateStream = extractSingleton(subtaskState.getRawKeyedState()); serializeKeyedStateHandle(keyedStateStream, dos); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java index 0578b787290d9..1fa5eb5b51484 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java @@ -18,11 +18,11 @@ package org.apache.flink.runtime.deployment; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.TaskInformation; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -64,7 +64,7 @@ public final class TaskDeploymentDescriptor implements Serializable { private final int targetSlotNumber; /** State handles for the sub task. */ - private final TaskStateHandles taskStateHandles; + private final TaskStateSnapshot taskStateHandles; public TaskDeploymentDescriptor( SerializedValue serializedJobInformation, @@ -74,7 +74,7 @@ public TaskDeploymentDescriptor( int subtaskIndex, int attemptNumber, int targetSlotNumber, - TaskStateHandles taskStateHandles, + TaskStateSnapshot taskStateHandles, Collection resultPartitionDeploymentDescriptors, Collection inputGateDeploymentDescriptors) { @@ -153,7 +153,7 @@ public Collection getInputGates() { return inputGates; } - public TaskStateHandles getTaskStateHandles() { + public TaskStateSnapshot getTaskStateHandles() { return taskStateHandles; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java index 9e9f7c4c719c4..203ee8547cf42 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java @@ -18,8 +18,6 @@ package org.apache.flink.runtime.execution; -import java.util.Map; -import java.util.concurrent.Future; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; @@ -28,7 +26,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; @@ -41,6 +39,9 @@ import org.apache.flink.runtime.state.internal.InternalKvState; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; +import java.util.Map; +import java.util.concurrent.Future; + /** * The Environment gives the code executed in a task access to the task's properties * (such as name, parallelism), the configurations, the data stream readers and writers, @@ -175,7 +176,7 @@ public interface Environment { * @param checkpointMetrics metrics for this checkpoint * @param subtaskState All state handles for the checkpointed state */ - void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, SubtaskState subtaskState); + void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, TaskStateSnapshot subtaskState); /** * Declines a checkpoint. This tells the checkpoint coordinator that this task will 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 bd5bc7f5da6cc..2074820b41dc3 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 @@ -24,6 +24,7 @@ import org.apache.flink.runtime.JobException; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor; @@ -41,7 +42,6 @@ import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.StackTraceSampleResponse; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.ExceptionUtils; @@ -133,7 +133,7 @@ public class Execution implements AccessExecution, Archiveable getPreferredLocations() { */ public Iterable getPreferredLocationsBasedOnState() { TaskManagerLocation priorLocation; - if (currentExecution.getTaskStateHandles() != null && (priorLocation = getLatestPriorLocation()) != null) { + if (currentExecution.getTaskStateSnapshot() != null && (priorLocation = getLatestPriorLocation()) != null) { return Collections.singleton(priorLocation); } else { @@ -719,7 +719,7 @@ void notifyStateTransition(Execution execution, ExecutionState newState, Throwab TaskDeploymentDescriptor createDeploymentDescriptor( ExecutionAttemptID executionId, SimpleSlot targetSlot, - TaskStateHandles taskStateHandles, + TaskStateSnapshot taskStateHandles, int attemptNumber) throws ExecutionGraphException { // Produced intermediate results diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/StatefulTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/StatefulTask.java index 0930011896353..00db01ffd2e04 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/StatefulTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/StatefulTask.java @@ -21,7 +21,7 @@ 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.state.TaskStateHandles; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; /** * This interface must be implemented by any invokable that has recoverable state and participates @@ -35,7 +35,7 @@ public interface StatefulTask { * * @param taskStateHandles All state handle for the task. */ - void setInitialState(TaskStateHandles taskStateHandles) throws Exception; + void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception; /** * This method is called to trigger a checkpoint, asynchronously by the checkpoint @@ -43,8 +43,8 @@ public interface StatefulTask { * *

This method is called for tasks that start the checkpoints by injecting the initial barriers, * i.e., the source tasks. In contrast, checkpoints on downstream operators, which are the result of - * receiving checkpoint barriers, invoke the {@link #triggerCheckpointOnBarrier(CheckpointMetaData, CheckpointMetrics)} - * method. + * receiving checkpoint barriers, invoke the + * {@link #triggerCheckpointOnBarrier(CheckpointMetaData, CheckpointOptions, CheckpointMetrics)} method. * * @param checkpointMetaData Meta data for about this checkpoint * @param checkpointOptions Options for performing this checkpoint 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 31036f6c8663d..25df19b51aaed 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 @@ -31,7 +31,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -96,6 +96,7 @@ import org.slf4j.Logger; import javax.annotation.Nullable; + import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -586,7 +587,7 @@ public void acknowledgeCheckpoint( final ExecutionAttemptID executionAttemptID, final long checkpointId, final CheckpointMetrics checkpointMetrics, - final SubtaskState checkpointState) { + final TaskStateSnapshot checkpointState) { final CheckpointCoordinator checkpointCoordinator = executionGraph.getCheckpointCoordinator(); final AcknowledgeCheckpoint ackMessage = diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/AcknowledgeCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/AcknowledgeCheckpoint.java index 9721c2cd6f306..65e3019951fcc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/AcknowledgeCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/checkpoint/AcknowledgeCheckpoint.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; /** @@ -36,7 +36,7 @@ public class AcknowledgeCheckpoint extends AbstractCheckpointMessage implements private static final long serialVersionUID = -7606214777192401493L; - private final SubtaskState subtaskState; + private final TaskStateSnapshot subtaskState; private final CheckpointMetrics checkpointMetrics; @@ -47,7 +47,7 @@ public AcknowledgeCheckpoint( ExecutionAttemptID taskExecutionId, long checkpointId, CheckpointMetrics checkpointMetrics, - SubtaskState subtaskState) { + TaskStateSnapshot subtaskState) { super(job, taskExecutionId, checkpointId); @@ -64,7 +64,7 @@ public AcknowledgeCheckpoint(JobID jobId, ExecutionAttemptID taskExecutionId, lo // properties // ------------------------------------------------------------------------ - public SubtaskState getSubtaskState() { + public TaskStateSnapshot getSubtaskState() { return subtaskState; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java index d82af7217a7cd..031d7c717284c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateInitializationContextImpl.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.state; -import org.apache.commons.io.IOUtils; import org.apache.flink.api.common.state.KeyedStateStore; import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.java.tuple.Tuple2; @@ -26,6 +25,8 @@ import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.util.Preconditions; +import org.apache.commons.io.IOUtils; + import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -139,6 +140,7 @@ public void close() { } private static Collection transform(Collection keyedStateHandles) { + if (keyedStateHandles == null) { return null; } @@ -146,13 +148,14 @@ private static Collection transform(Collection keyGroupsStateHandles = new ArrayList<>(); for (KeyedStateHandle keyedStateHandle : keyedStateHandles) { - if (! (keyedStateHandle instanceof KeyGroupsStateHandle)) { + + if (keyedStateHandle instanceof KeyGroupsStateHandle) { + keyGroupsStateHandles.add((KeyGroupsStateHandle) keyedStateHandle); + } else if (keyedStateHandle != null) { throw new IllegalStateException("Unexpected state handle type, " + "expected: " + KeyGroupsStateHandle.class + ", but found: " + keyedStateHandle.getClass() + "."); } - - keyGroupsStateHandles.add((KeyGroupsStateHandle) keyedStateHandle); } return keyGroupsStateHandles; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java deleted file mode 100644 index 2fde5485049f9..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskStateHandles.java +++ /dev/null @@ -1,172 +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.state; - -import org.apache.flink.runtime.checkpoint.SubtaskState; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -/** - * This class encapsulates all state handles for a task. - */ -public class TaskStateHandles implements Serializable { - - public static final TaskStateHandles EMPTY = new TaskStateHandles(); - - private static final long serialVersionUID = 267686583583579359L; - - /** - * State handle with the (non-partitionable) legacy operator state - * - * @deprecated Non-repartitionable operator state that has been deprecated. - * Can be removed when we remove the APIs for non-repartitionable operator state. - */ - @Deprecated - private final ChainedStateHandle legacyOperatorState; - - /** Collection of handles which represent the managed keyed state of the head operator */ - private final Collection managedKeyedState; - - /** Collection of handles which represent the raw/streamed keyed state of the head operator */ - private final Collection rawKeyedState; - - /** Outer list represents the operator chain, each collection holds handles for managed state of a single operator */ - private final List> managedOperatorState; - - /** Outer list represents the operator chain, each collection holds handles for raw/streamed state of a single operator */ - private final List> rawOperatorState; - - public TaskStateHandles() { - this(null, null, null, null, null); - } - - public TaskStateHandles(SubtaskState checkpointStateHandles) { - this(checkpointStateHandles.getLegacyOperatorState(), - transform(checkpointStateHandles.getManagedOperatorState()), - transform(checkpointStateHandles.getRawOperatorState()), - transform(checkpointStateHandles.getManagedKeyedState()), - transform(checkpointStateHandles.getRawKeyedState())); - } - - public TaskStateHandles( - ChainedStateHandle legacyOperatorState, - List> managedOperatorState, - List> rawOperatorState, - Collection managedKeyedState, - Collection rawKeyedState) { - - this.legacyOperatorState = legacyOperatorState; - this.managedKeyedState = managedKeyedState; - this.rawKeyedState = rawKeyedState; - this.managedOperatorState = managedOperatorState; - this.rawOperatorState = rawOperatorState; - } - - /** - * @deprecated Non-repartitionable operator state that has been deprecated. - * Can be removed when we remove the APIs for non-repartitionable operator state. - */ - @Deprecated - public ChainedStateHandle getLegacyOperatorState() { - return legacyOperatorState; - } - - public Collection getManagedKeyedState() { - return managedKeyedState; - } - - public Collection getRawKeyedState() { - return rawKeyedState; - } - - public List> getRawOperatorState() { - return rawOperatorState; - } - - public List> getManagedOperatorState() { - return managedOperatorState; - } - - private static List> transform(ChainedStateHandle in) { - if (null == in) { - return Collections.emptyList(); - } - List> out = new ArrayList<>(in.getLength()); - for (int i = 0; i < in.getLength(); ++i) { - OperatorStateHandle osh = in.get(i); - out.add(osh != null ? Collections.singletonList(osh) : null); - } - return out; - } - - private static List transform(T in) { - return in == null ? Collections.emptyList() : Collections.singletonList(in); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - TaskStateHandles that = (TaskStateHandles) o; - - if (legacyOperatorState != null ? - !legacyOperatorState.equals(that.legacyOperatorState) - : that.legacyOperatorState != null) { - return false; - } - if (managedKeyedState != null ? - !managedKeyedState.equals(that.managedKeyedState) - : that.managedKeyedState != null) { - return false; - } - if (rawKeyedState != null ? - !rawKeyedState.equals(that.rawKeyedState) - : that.rawKeyedState != null) { - return false; - } - - if (rawOperatorState != null ? - !rawOperatorState.equals(that.rawOperatorState) - : that.rawOperatorState != null) { - return false; - } - return managedOperatorState != null ? - managedOperatorState.equals(that.managedOperatorState) - : that.managedOperatorState == null; - } - - @Override - public int hashCode() { - int result = legacyOperatorState != null ? legacyOperatorState.hashCode() : 0; - result = 31 * result + (managedKeyedState != null ? managedKeyedState.hashCode() : 0); - result = 31 * result + (rawKeyedState != null ? rawKeyedState.hashCode() : 0); - result = 31 * result + (managedOperatorState != null ? managedOperatorState.hashCode() : 0); - result = 31 * result + (rawOperatorState != null ? rawOperatorState.hashCode() : 0); - return result; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java index bf6016126af1d..aba8bda191825 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcCheckpointResponder.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.util.Preconditions; @@ -40,7 +40,7 @@ public void acknowledgeCheckpoint( ExecutionAttemptID executionAttemptID, long checkpointId, CheckpointMetrics checkpointMetrics, - SubtaskState subtaskState) { + TaskStateSnapshot subtaskState) { checkpointCoordinatorGateway.acknowledgeCheckpoint( jobID, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java index ad0df7151c20a..e9f600d672abc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; @@ -44,7 +44,7 @@ public void acknowledgeCheckpoint( ExecutionAttemptID executionAttemptID, long checkpointId, CheckpointMetrics checkpointMetrics, - SubtaskState checkpointStateHandles) { + TaskStateSnapshot checkpointStateHandles) { AcknowledgeCheckpoint message = new AcknowledgeCheckpoint( jobID, executionAttemptID, checkpointId, checkpointMetrics, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java index cc66a3f283160..b3584a6dfc987 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; /** @@ -47,7 +47,7 @@ void acknowledgeCheckpoint( ExecutionAttemptID executionAttemptID, long checkpointId, CheckpointMetrics checkpointMetrics, - SubtaskState subtaskState); + TaskStateSnapshot subtaskState); /** * Declines the given checkpoint. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java index 788a59090d309..92b58868d666f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -245,7 +245,7 @@ public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpoin public void acknowledgeCheckpoint( long checkpointId, CheckpointMetrics checkpointMetrics, - SubtaskState checkpointStateHandles) { + TaskStateSnapshot checkpointStateHandles) { checkpointResponder.acknowledgeCheckpoint( jobId, executionId, checkpointId, checkpointMetrics, 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 596d36519f3ba..04cb99038dc4e 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 @@ -34,6 +34,7 @@ import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineTaskNotCheckpointingException; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineTaskNotReadyException; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -67,16 +68,17 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.WrappingRuntimeException; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; import javax.annotation.Nullable; + import java.io.IOException; import java.net.URL; import java.util.Collection; @@ -250,7 +252,7 @@ public class Task implements Runnable, TaskActions { * The handles to the states that the task was initialized with. Will be set * to null after the initialization, to be memory friendly. */ - private volatile TaskStateHandles taskStateHandles; + private volatile TaskStateSnapshot taskStateHandles; /** Initialized from the Flink configuration. May also be set at the ExecutionConfig */ private long taskCancellationInterval; @@ -272,7 +274,7 @@ public Task( Collection resultPartitionDeploymentDescriptors, Collection inputGateDeploymentDescriptors, int targetSlotNumber, - TaskStateHandles taskStateHandles, + TaskStateSnapshot taskStateHandles, MemoryManager memManager, IOManager ioManager, NetworkEnvironment networkEnvironment, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index 344b34093d9b7..88b95f5959fd7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -23,14 +23,15 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobStatus; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.util.TestLogger; + import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; @@ -42,8 +43,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.anyInt; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -89,29 +90,26 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { assertFalse(pendingCheckpoint.isDiscarded()); final long checkpointId = coord.getPendingCheckpoints().keySet().iterator().next(); - - SubtaskState subtaskState = mock(SubtaskState.class); + StreamStateHandle legacyHandle = mock(StreamStateHandle.class); - ChainedStateHandle chainedLegacyHandle = mock(ChainedStateHandle.class); - when(chainedLegacyHandle.get(anyInt())).thenReturn(legacyHandle); - when(subtaskState.getLegacyOperatorState()).thenReturn(chainedLegacyHandle); + KeyedStateHandle managedKeyedHandle = mock(KeyedStateHandle.class); + KeyedStateHandle rawKeyedHandle = mock(KeyedStateHandle.class); + OperatorStateHandle managedOpHandle = mock(OperatorStateHandle.class); + OperatorStateHandle rawOpHandle = mock(OperatorStateHandle.class); - OperatorStateHandle managedHandle = mock(OperatorStateHandle.class); - ChainedStateHandle chainedManagedHandle = mock(ChainedStateHandle.class); - when(chainedManagedHandle.get(anyInt())).thenReturn(managedHandle); - when(subtaskState.getManagedOperatorState()).thenReturn(chainedManagedHandle); + final OperatorSubtaskState operatorSubtaskState = spy(new OperatorSubtaskState( + legacyHandle, + managedOpHandle, + rawOpHandle, + managedKeyedHandle, + rawKeyedHandle)); - OperatorStateHandle rawHandle = mock(OperatorStateHandle.class); - ChainedStateHandle chainedRawHandle = mock(ChainedStateHandle.class); - when(chainedRawHandle.get(anyInt())).thenReturn(rawHandle); - when(subtaskState.getRawOperatorState()).thenReturn(chainedRawHandle); + TaskStateSnapshot subtaskState = spy(new TaskStateSnapshot()); + subtaskState.putSubtaskStateByOperatorID(new OperatorID(), operatorSubtaskState); + + when(subtaskState.getSubtaskStateByOperatorID(OperatorID.fromJobVertexID(vertex.getJobvertexId()))).thenReturn(operatorSubtaskState); - KeyedStateHandle managedKeyedHandle = mock(KeyedStateHandle.class); - when(subtaskState.getRawKeyedState()).thenReturn(managedKeyedHandle); - KeyedStateHandle managedRawHandle = mock(KeyedStateHandle.class); - when(subtaskState.getManagedKeyedState()).thenReturn(managedRawHandle); - AcknowledgeCheckpoint acknowledgeMessage = new AcknowledgeCheckpoint(jid, executionAttemptId, checkpointId, new CheckpointMetrics(), subtaskState); try { @@ -126,11 +124,12 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { assertTrue(pendingCheckpoint.isDiscarded()); // make sure that the subtask state has been discarded after we could not complete it. - verify(subtaskState.getLegacyOperatorState().get(0)).discardState(); - verify(subtaskState.getManagedOperatorState().get(0)).discardState(); - verify(subtaskState.getRawOperatorState().get(0)).discardState(); - verify(subtaskState.getManagedKeyedState()).discardState(); - verify(subtaskState.getRawKeyedState()).discardState(); + verify(operatorSubtaskState).discardState(); + verify(operatorSubtaskState.getLegacyOperatorState()).discardState(); + verify(operatorSubtaskState.getManagedOperatorState().iterator().next()).discardState(); + verify(operatorSubtaskState.getRawOperatorState().iterator().next()).discardState(); + verify(operatorSubtaskState.getManagedKeyedState().iterator().next()).discardState(); + verify(operatorSubtaskState.getRawKeyedState().iterator().next()).discardState(); } private static final class FailingCompletedCheckpointStore implements CompletedCheckpointStore { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index cb92df66b81bc..d9af8797b7b17 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -44,7 +44,6 @@ import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.state.filesystem.FileStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -93,7 +92,6 @@ import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyLong; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -102,7 +100,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import static org.mockito.Mockito.withSettings; /** * Tests for the checkpoint coordinator. @@ -555,31 +552,29 @@ public void testTriggerAndConfirmSimpleCheckpoint() { assertFalse(checkpoint.isDiscarded()); assertFalse(checkpoint.isFullyAcknowledged()); - OperatorID opID1 = OperatorID.fromJobVertexID(vertex1.getJobvertexId()); - OperatorID opID2 = OperatorID.fromJobVertexID(vertex2.getJobvertexId()); - - Map operatorStates = checkpoint.getOperatorStates(); - - operatorStates.put(opID1, new SpyInjectingOperatorState( - opID1, vertex1.getTotalNumberOfParallelSubtasks(), vertex1.getMaxParallelism())); - operatorStates.put(opID2, new SpyInjectingOperatorState( - opID2, vertex2.getTotalNumberOfParallelSubtasks(), vertex2.getMaxParallelism())); - // check that the vertices received the trigger checkpoint message { verify(vertex1.getCurrentExecutionAttempt(), times(1)).triggerCheckpoint(eq(checkpointId), eq(timestamp), any(CheckpointOptions.class)); verify(vertex2.getCurrentExecutionAttempt(), times(1)).triggerCheckpoint(eq(checkpointId), eq(timestamp), any(CheckpointOptions.class)); } + OperatorID opID1 = OperatorID.fromJobVertexID(vertex1.getJobvertexId()); + OperatorID opID2 = OperatorID.fromJobVertexID(vertex2.getJobvertexId()); + TaskStateSnapshot taskOperatorSubtaskStates1 = mock(TaskStateSnapshot.class); + TaskStateSnapshot taskOperatorSubtaskStates2 = mock(TaskStateSnapshot.class); + OperatorSubtaskState subtaskState1 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState2 = mock(OperatorSubtaskState.class); + when(taskOperatorSubtaskStates1.getSubtaskStateByOperatorID(opID1)).thenReturn(subtaskState1); + when(taskOperatorSubtaskStates2.getSubtaskStateByOperatorID(opID2)).thenReturn(subtaskState2); + // acknowledge from one of the tasks - AcknowledgeCheckpoint acknowledgeCheckpoint1 = new AcknowledgeCheckpoint(jid, attemptID2, checkpointId, new CheckpointMetrics(), mock(SubtaskState.class)); + AcknowledgeCheckpoint acknowledgeCheckpoint1 = new AcknowledgeCheckpoint(jid, attemptID2, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStates2); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint1); - OperatorSubtaskState subtaskState2 = operatorStates.get(opID2).getState(vertex2.getParallelSubtaskIndex()); assertEquals(1, checkpoint.getNumberOfAcknowledgedTasks()); assertEquals(1, checkpoint.getNumberOfNonAcknowledgedTasks()); assertFalse(checkpoint.isDiscarded()); assertFalse(checkpoint.isFullyAcknowledged()); - verify(subtaskState2, never()).registerSharedStates(any(SharedStateRegistry.class)); + verify(taskOperatorSubtaskStates2, never()).registerSharedStates(any(SharedStateRegistry.class)); // acknowledge the same task again (should not matter) coord.receiveAcknowledgeMessage(acknowledgeCheckpoint1); @@ -588,8 +583,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { verify(subtaskState2, never()).registerSharedStates(any(SharedStateRegistry.class)); // acknowledge the other task. - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId, new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState subtaskState1 = operatorStates.get(opID1).getState(vertex1.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStates1)); // the checkpoint is internally converted to a successful checkpoint and the // pending checkpoint object is disposed @@ -628,9 +622,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() { long checkpointIdNew = coord.getPendingCheckpoints().entrySet().iterator().next().getKey(); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew)); - subtaskState1 = operatorStates.get(opID1).getState(vertex1.getParallelSubtaskIndex()); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointIdNew)); - subtaskState2 = operatorStates.get(opID2).getState(vertex2.getParallelSubtaskIndex()); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(1, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -852,18 +844,20 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { OperatorID opID2 = OperatorID.fromJobVertexID(ackVertex2.getJobvertexId()); OperatorID opID3 = OperatorID.fromJobVertexID(ackVertex3.getJobvertexId()); - Map operatorStates1 = pending1.getOperatorStates(); + TaskStateSnapshot taskOperatorSubtaskStates1_1 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates1_2 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates1_3 = spy(new TaskStateSnapshot()); - operatorStates1.put(opID1, new SpyInjectingOperatorState( - opID1, ackVertex1.getTotalNumberOfParallelSubtasks(), ackVertex1.getMaxParallelism())); - operatorStates1.put(opID2, new SpyInjectingOperatorState( - opID2, ackVertex2.getTotalNumberOfParallelSubtasks(), ackVertex2.getMaxParallelism())); - operatorStates1.put(opID3, new SpyInjectingOperatorState( - opID3, ackVertex3.getTotalNumberOfParallelSubtasks(), ackVertex3.getMaxParallelism())); + OperatorSubtaskState subtaskState1_1 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState1_2 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState1_3 = mock(OperatorSubtaskState.class); + taskOperatorSubtaskStates1_1.putSubtaskStateByOperatorID(opID1, subtaskState1_1); + taskOperatorSubtaskStates1_2.putSubtaskStateByOperatorID(opID2, subtaskState1_2); + taskOperatorSubtaskStates1_3.putSubtaskStateByOperatorID(opID3, subtaskState1_3); // acknowledge one of the three tasks - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId1, new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState subtaskState1_2 = operatorStates1.get(opID2).getState(ackVertex2.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates1_2)); + // start the second checkpoint // trigger the first checkpoint. this should succeed assertTrue(coord.triggerCheckpoint(timestamp2, false)); @@ -880,14 +874,17 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { } long checkpointId2 = pending2.getCheckpointId(); - Map operatorStates2 = pending2.getOperatorStates(); + TaskStateSnapshot taskOperatorSubtaskStates2_1 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates2_2 = spy(new TaskStateSnapshot()); + TaskStateSnapshot taskOperatorSubtaskStates2_3 = spy(new TaskStateSnapshot()); + + OperatorSubtaskState subtaskState2_1 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState2_2 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState2_3 = mock(OperatorSubtaskState.class); - operatorStates2.put(opID1, new SpyInjectingOperatorState( - opID1, ackVertex1.getTotalNumberOfParallelSubtasks(), ackVertex1.getMaxParallelism())); - operatorStates2.put(opID2, new SpyInjectingOperatorState( - opID2, ackVertex2.getTotalNumberOfParallelSubtasks(), ackVertex2.getMaxParallelism())); - operatorStates2.put(opID3, new SpyInjectingOperatorState( - opID3, ackVertex3.getTotalNumberOfParallelSubtasks(), ackVertex3.getMaxParallelism())); + taskOperatorSubtaskStates2_1.putSubtaskStateByOperatorID(opID1, subtaskState2_1); + taskOperatorSubtaskStates2_2.putSubtaskStateByOperatorID(opID2, subtaskState2_2); + taskOperatorSubtaskStates2_3.putSubtaskStateByOperatorID(opID3, subtaskState2_3); // trigger messages should have been sent verify(triggerVertex1.getCurrentExecutionAttempt(), times(1)).triggerCheckpoint(eq(checkpointId2), eq(timestamp2), any(CheckpointOptions.class)); @@ -896,17 +893,13 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { // we acknowledge one more task from the first checkpoint and the second // checkpoint completely. The second checkpoint should then subsume the first checkpoint - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId2, new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState subtaskState2_3 = operatorStates2.get(opID3).getState(ackVertex3.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates2_3)); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId2, new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState subtaskState2_1 = operatorStates2.get(opID1).getState(ackVertex1.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates2_1)); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId1, new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState subtaskState1_1 = operatorStates1.get(opID1).getState(ackVertex1.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates1_1)); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2, new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState subtaskState2_2 = operatorStates2.get(opID2).getState(ackVertex2.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID2, checkpointId2, new CheckpointMetrics(), taskOperatorSubtaskStates2_2)); // now, the second checkpoint should be confirmed, and the first discarded // actually both pending checkpoints are discarded, and the second has been transformed @@ -938,8 +931,7 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { verify(commitVertex.getCurrentExecutionAttempt(), times(1)).notifyCheckpointComplete(eq(checkpointId2), eq(timestamp2)); // send the last remaining ack for the first checkpoint. This should not do anything - SubtaskState subtaskState1_3 = mock(SubtaskState.class); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId1, new CheckpointMetrics(), subtaskState1_3)); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID3, checkpointId1, new CheckpointMetrics(), taskOperatorSubtaskStates1_3)); verify(subtaskState1_3, times(1)).discardState(); coord.shutdown(JobStatus.FINISHED); @@ -1005,13 +997,11 @@ public void testCheckpointTimeoutIsolated() { OperatorID opID1 = OperatorID.fromJobVertexID(ackVertex1.getJobvertexId()); - Map operatorStates = checkpoint.getOperatorStates(); + TaskStateSnapshot taskOperatorSubtaskStates1 = spy(new TaskStateSnapshot()); + OperatorSubtaskState subtaskState1 = mock(OperatorSubtaskState.class); + taskOperatorSubtaskStates1.putSubtaskStateByOperatorID(opID1, subtaskState1); - operatorStates.put(opID1, new SpyInjectingOperatorState( - opID1, ackVertex1.getTotalNumberOfParallelSubtasks(), ackVertex1.getMaxParallelism())); - - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpoint.getCheckpointId(), new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState subtaskState = operatorStates.get(opID1).getState(ackVertex1.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, ackAttemptID1, checkpoint.getCheckpointId(), new CheckpointMetrics(), taskOperatorSubtaskStates1)); // wait until the checkpoint must have expired. // we check every 250 msecs conservatively for 5 seconds @@ -1029,7 +1019,7 @@ public void testCheckpointTimeoutIsolated() { assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); // validate that the received states have been discarded - verify(subtaskState, times(1)).discardState(); + verify(subtaskState1, times(1)).discardState(); // no confirm message must have been sent verify(commitVertex.getCurrentExecutionAttempt(), times(0)).notifyCheckpointComplete(anyLong(), anyLong()); @@ -1147,26 +1137,18 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { long checkpointId = pendingCheckpoint.getCheckpointId(); OperatorID opIDtrigger = OperatorID.fromJobVertexID(triggerVertex.getJobvertexId()); - OperatorID opID1 = OperatorID.fromJobVertexID(ackVertex1.getJobvertexId()); - OperatorID opID2 = OperatorID.fromJobVertexID(ackVertex2.getJobvertexId()); - - Map operatorStates = pendingCheckpoint.getOperatorStates(); - operatorStates.put(opIDtrigger, new SpyInjectingOperatorState( - opIDtrigger, triggerVertex.getTotalNumberOfParallelSubtasks(), triggerVertex.getMaxParallelism())); - operatorStates.put(opID1, new SpyInjectingOperatorState( - opID1, ackVertex1.getTotalNumberOfParallelSubtasks(), ackVertex1.getMaxParallelism())); - operatorStates.put(opID2, new SpyInjectingOperatorState( - opID2, ackVertex2.getTotalNumberOfParallelSubtasks(), ackVertex2.getMaxParallelism())); + TaskStateSnapshot taskOperatorSubtaskStatesTrigger = spy(new TaskStateSnapshot()); + OperatorSubtaskState subtaskStateTrigger = mock(OperatorSubtaskState.class); + taskOperatorSubtaskStatesTrigger.putSubtaskStateByOperatorID(opIDtrigger, subtaskStateTrigger); // acknowledge the first trigger vertex - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jobId, triggerAttemptId, checkpointId, new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState storedTriggerSubtaskState = operatorStates.get(opIDtrigger).getState(triggerVertex.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jobId, triggerAttemptId, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStatesTrigger)); // verify that the subtask state has not been discarded - verify(storedTriggerSubtaskState, never()).discardState(); + verify(subtaskStateTrigger, never()).discardState(); - SubtaskState unknownSubtaskState = mock(SubtaskState.class); + TaskStateSnapshot unknownSubtaskState = mock(TaskStateSnapshot.class); // receive an acknowledge message for an unknown vertex coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jobId, new ExecutionAttemptID(), checkpointId, new CheckpointMetrics(), unknownSubtaskState)); @@ -1174,7 +1156,7 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { // we should discard acknowledge messages from an unknown vertex belonging to our job verify(unknownSubtaskState, times(1)).discardState(); - SubtaskState differentJobSubtaskState = mock(SubtaskState.class); + TaskStateSnapshot differentJobSubtaskState = mock(TaskStateSnapshot.class); // receive an acknowledge message from an unknown job coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(new JobID(), new ExecutionAttemptID(), checkpointId, new CheckpointMetrics(), differentJobSubtaskState)); @@ -1183,22 +1165,22 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { verify(differentJobSubtaskState, never()).discardState(); // duplicate acknowledge message for the trigger vertex - SubtaskState triggerSubtaskState = mock(SubtaskState.class); + TaskStateSnapshot triggerSubtaskState = mock(TaskStateSnapshot.class); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jobId, triggerAttemptId, checkpointId, new CheckpointMetrics(), triggerSubtaskState)); // duplicate acknowledge messages for a known vertex should not trigger discarding the state verify(triggerSubtaskState, never()).discardState(); // let the checkpoint fail at the first ack vertex - reset(storedTriggerSubtaskState); + reset(subtaskStateTrigger); coord.receiveDeclineMessage(new DeclineCheckpoint(jobId, ackAttemptId1, checkpointId)); assertTrue(pendingCheckpoint.isDiscarded()); // check that we've cleaned up the already acknowledged state - verify(storedTriggerSubtaskState, times(1)).discardState(); + verify(subtaskStateTrigger, times(1)).discardState(); - SubtaskState ackSubtaskState = mock(SubtaskState.class); + TaskStateSnapshot ackSubtaskState = mock(TaskStateSnapshot.class); // late acknowledge message from the second ack vertex coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jobId, ackAttemptId2, checkpointId, new CheckpointMetrics(), ackSubtaskState)); @@ -1213,7 +1195,7 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { // we should not interfere with different jobs verify(differentJobSubtaskState, never()).discardState(); - SubtaskState unknownSubtaskState2 = mock(SubtaskState.class); + TaskStateSnapshot unknownSubtaskState2 = mock(TaskStateSnapshot.class); // receive an acknowledge message for an unknown vertex coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jobId, new ExecutionAttemptID(), checkpointId, new CheckpointMetrics(), unknownSubtaskState2)); @@ -1470,18 +1452,16 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { OperatorID opID1 = OperatorID.fromJobVertexID(vertex1.getJobvertexId()); OperatorID opID2 = OperatorID.fromJobVertexID(vertex2.getJobvertexId()); - - Map operatorStates = pending.getOperatorStates(); - - operatorStates.put(opID1, new SpyInjectingOperatorState( - opID1, vertex1.getTotalNumberOfParallelSubtasks(), vertex1.getMaxParallelism())); - operatorStates.put(opID2, new SpyInjectingOperatorState( - opID2, vertex2.getTotalNumberOfParallelSubtasks(), vertex1.getMaxParallelism())); + TaskStateSnapshot taskOperatorSubtaskStates1 = mock(TaskStateSnapshot.class); + TaskStateSnapshot taskOperatorSubtaskStates2 = mock(TaskStateSnapshot.class); + OperatorSubtaskState subtaskState1 = mock(OperatorSubtaskState.class); + OperatorSubtaskState subtaskState2 = mock(OperatorSubtaskState.class); + when(taskOperatorSubtaskStates1.getSubtaskStateByOperatorID(opID1)).thenReturn(subtaskState1); + when(taskOperatorSubtaskStates2.getSubtaskStateByOperatorID(opID2)).thenReturn(subtaskState2); // acknowledge from one of the tasks - AcknowledgeCheckpoint acknowledgeCheckpoint2 = new AcknowledgeCheckpoint(jid, attemptID2, checkpointId, new CheckpointMetrics(), mock(SubtaskState.class)); + AcknowledgeCheckpoint acknowledgeCheckpoint2 = new AcknowledgeCheckpoint(jid, attemptID2, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStates2); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint2); - OperatorSubtaskState subtaskState2 = operatorStates.get(opID2).getState(vertex2.getParallelSubtaskIndex()); assertEquals(1, pending.getNumberOfAcknowledgedTasks()); assertEquals(1, pending.getNumberOfNonAcknowledgedTasks()); assertFalse(pending.isDiscarded()); @@ -1495,8 +1475,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { assertFalse(savepointFuture.isDone()); // acknowledge the other task. - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId, new CheckpointMetrics(), mock(SubtaskState.class))); - OperatorSubtaskState subtaskState1 = operatorStates.get(opID1).getState(vertex1.getParallelSubtaskIndex()); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointId, new CheckpointMetrics(), taskOperatorSubtaskStates1)); // the checkpoint is internally converted to a successful checkpoint and the // pending checkpoint object is disposed @@ -1536,9 +1515,6 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID1, checkpointIdNew)); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, attemptID2, checkpointIdNew)); - subtaskState1 = operatorStates.get(opID1).getState(vertex1.getParallelSubtaskIndex()); - subtaskState2 = operatorStates.get(opID2).getState(vertex2.getParallelSubtaskIndex()); - assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -2037,20 +2013,8 @@ public void testRestoreLatestCheckpointedState() throws Exception { List keyGroupPartitions1 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); List keyGroupPartitions2 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); - PendingCheckpoint pending = coord.getPendingCheckpoints().get(checkpointId); - - OperatorID opID1 = OperatorID.fromJobVertexID(jobVertexID1); - OperatorID opID2 = OperatorID.fromJobVertexID(jobVertexID2); - - Map operatorStates = pending.getOperatorStates(); - - operatorStates.put(opID1, new SpyInjectingOperatorState( - opID1, jobVertex1.getParallelism(), jobVertex1.getMaxParallelism())); - operatorStates.put(opID2, new SpyInjectingOperatorState( - opID2, jobVertex2.getParallelism(), jobVertex2.getMaxParallelism())); - for (int index = 0; index < jobVertex1.getParallelism(); index++) { - SubtaskState subtaskState = mockSubtaskState(jobVertexID1, index, keyGroupPartitions1.get(index)); + TaskStateSnapshot subtaskState = mockSubtaskState(jobVertexID1, index, keyGroupPartitions1.get(index)); AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( jid, @@ -2063,7 +2027,7 @@ public void testRestoreLatestCheckpointedState() throws Exception { } for (int index = 0; index < jobVertex2.getParallelism(); index++) { - SubtaskState subtaskState = mockSubtaskState(jobVertexID2, index, keyGroupPartitions2.get(index)); + TaskStateSnapshot subtaskState = mockSubtaskState(jobVertexID2, index, keyGroupPartitions2.get(index)); AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( jid, @@ -2165,30 +2129,34 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws List keyGroupPartitions2 = StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); for (int index = 0; index < jobVertex1.getParallelism(); index++) { - ChainedStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index); + StreamStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index); KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), false); - SubtaskState checkpointStateHandles = new SubtaskState(valueSizeTuple, null, null, keyGroupState, null); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(valueSizeTuple, null, null, keyGroupState, null); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState); AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( jid, jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), checkpointId, new CheckpointMetrics(), - checkpointStateHandles); + taskOperatorSubtaskStates); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint); } for (int index = 0; index < jobVertex2.getParallelism(); index++) { - ChainedStateHandle valueSizeTuple = generateStateForVertex(jobVertexID2, index); + StreamStateHandle valueSizeTuple = generateStateForVertex(jobVertexID2, index); KeyGroupsStateHandle keyGroupState = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), false); - SubtaskState checkpointStateHandles = new SubtaskState(valueSizeTuple, null, null, keyGroupState, null); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(valueSizeTuple, null, null, keyGroupState, null); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState); AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( jid, jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), checkpointId, new CheckpointMetrics(), - checkpointStateHandles); + taskOperatorSubtaskStates); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint); } @@ -2284,17 +2252,20 @@ public void testRestoreLatestCheckpointFailureWhenParallelismChanges() throws Ex StateAssignmentOperation.createKeyGroupPartitions(maxParallelism2, parallelism2); for (int index = 0; index < jobVertex1.getParallelism(); index++) { - ChainedStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index); + StreamStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index); KeyGroupsStateHandle keyGroupState = generateKeyGroupState( jobVertexID1, keyGroupPartitions1.get(index), false); - SubtaskState checkpointStateHandles = new SubtaskState(valueSizeTuple, null, null, keyGroupState, null); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(valueSizeTuple, null, null, keyGroupState, null); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState); + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( jid, jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), checkpointId, new CheckpointMetrics(), - checkpointStateHandles); + taskOperatorSubtaskStates); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint); } @@ -2302,17 +2273,19 @@ public void testRestoreLatestCheckpointFailureWhenParallelismChanges() throws Ex for (int index = 0; index < jobVertex2.getParallelism(); index++) { - ChainedStateHandle state = generateStateForVertex(jobVertexID2, index); + StreamStateHandle state = generateStateForVertex(jobVertexID2, index); KeyGroupsStateHandle keyGroupState = generateKeyGroupState( jobVertexID2, keyGroupPartitions2.get(index), false); - SubtaskState checkpointStateHandles = new SubtaskState(state, null, null, keyGroupState, null); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(state, null, null, keyGroupState, null); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState); AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( jid, jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), checkpointId, new CheckpointMetrics(), - checkpointStateHandles); + taskOperatorSubtaskStates); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint); } @@ -2438,18 +2411,21 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s //vertex 1 for (int index = 0; index < jobVertex1.getParallelism(); index++) { - ChainedStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index); - ChainedStateHandle opStateBackend = generateChainedPartitionableStateHandle(jobVertexID1, index, 2, 8, false); + StreamStateHandle valueSizeTuple = generateStateForVertex(jobVertexID1, index); + OperatorStateHandle opStateBackend = generatePartitionableStateHandle(jobVertexID1, index, 2, 8, false); KeyGroupsStateHandle keyedStateBackend = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), false); KeyGroupsStateHandle keyedStateRaw = generateKeyGroupState(jobVertexID1, keyGroupPartitions1.get(index), true); - SubtaskState checkpointStateHandles = new SubtaskState(valueSizeTuple, opStateBackend, null, keyedStateBackend, keyedStateRaw); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(valueSizeTuple, opStateBackend, null, keyedStateBackend, keyedStateRaw); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID1), operatorSubtaskState); + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( jid, jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), checkpointId, new CheckpointMetrics(), - checkpointStateHandles); + taskOperatorSubtaskStates); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint); } @@ -2460,19 +2436,21 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s for (int index = 0; index < jobVertex2.getParallelism(); index++) { KeyGroupsStateHandle keyedStateBackend = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), false); KeyGroupsStateHandle keyedStateRaw = generateKeyGroupState(jobVertexID2, keyGroupPartitions2.get(index), true); - ChainedStateHandle opStateBackend = generateChainedPartitionableStateHandle(jobVertexID2, index, 2, 8, false); - ChainedStateHandle opStateRaw = generateChainedPartitionableStateHandle(jobVertexID2, index, 2, 8, true); - expectedOpStatesBackend.add(opStateBackend); - expectedOpStatesRaw.add(opStateRaw); - SubtaskState checkpointStateHandles = - new SubtaskState(new ChainedStateHandle<>( - Collections.singletonList(null)), opStateBackend, opStateRaw, keyedStateBackend, keyedStateRaw); + OperatorStateHandle opStateBackend = generatePartitionableStateHandle(jobVertexID2, index, 2, 8, false); + OperatorStateHandle opStateRaw = generatePartitionableStateHandle(jobVertexID2, index, 2, 8, true); + expectedOpStatesBackend.add(new ChainedStateHandle<>(Collections.singletonList(opStateBackend))); + expectedOpStatesRaw.add(new ChainedStateHandle<>(Collections.singletonList(opStateRaw))); + + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState(null, opStateBackend, opStateRaw, keyedStateBackend, keyedStateRaw); + TaskStateSnapshot taskOperatorSubtaskStates = new TaskStateSnapshot(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID2), operatorSubtaskState); + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( jid, jobVertex2.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), checkpointId, new CheckpointMetrics(), - checkpointStateHandles); + taskOperatorSubtaskStates); coord.receiveAcknowledgeMessage(acknowledgeCheckpoint); } @@ -2506,27 +2484,37 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s List>> actualOpStatesBackend = new ArrayList<>(newJobVertex2.getParallelism()); List>> actualOpStatesRaw = new ArrayList<>(newJobVertex2.getParallelism()); for (int i = 0; i < newJobVertex2.getParallelism(); i++) { - KeyGroupsStateHandle originalKeyedStateBackend = generateKeyGroupState(jobVertexID2, newKeyGroupPartitions2.get(i), false); - KeyGroupsStateHandle originalKeyedStateRaw = generateKeyGroupState(jobVertexID2, newKeyGroupPartitions2.get(i), true); - TaskStateHandles taskStateHandles = newJobVertex2.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateHandles(); + List operatorIDs = newJobVertex2.getOperatorIDs(); - ChainedStateHandle operatorState = taskStateHandles.getLegacyOperatorState(); - List> opStateBackend = taskStateHandles.getManagedOperatorState(); - List> opStateRaw = taskStateHandles.getRawOperatorState(); - Collection keyedStateBackend = taskStateHandles.getManagedKeyedState(); - Collection keyGroupStateRaw = taskStateHandles.getRawKeyedState(); + KeyGroupsStateHandle originalKeyedStateBackend = generateKeyGroupState(jobVertexID2, newKeyGroupPartitions2.get(i), false); + KeyGroupsStateHandle originalKeyedStateRaw = generateKeyGroupState(jobVertexID2, newKeyGroupPartitions2.get(i), true); - actualOpStatesBackend.add(opStateBackend); - actualOpStatesRaw.add(opStateRaw); - // the 'non partition state' is not null because it is recombined. - assertNotNull(operatorState); - for (int index = 0; index < operatorState.getLength(); index++) { - assertNull(operatorState.get(index)); + TaskStateSnapshot taskStateHandles = newJobVertex2.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateSnapshot(); + + final int headOpIndex = operatorIDs.size() - 1; + List> allParallelManagedOpStates = new ArrayList<>(operatorIDs.size()); + List> allParallelRawOpStates = new ArrayList<>(operatorIDs.size()); + + for (int idx = 0; idx < operatorIDs.size(); ++idx) { + OperatorID operatorID = operatorIDs.get(idx); + OperatorSubtaskState opState = taskStateHandles.getSubtaskStateByOperatorID(operatorID); + Assert.assertNull(opState.getLegacyOperatorState()); + Collection opStateBackend = opState.getManagedOperatorState(); + Collection opStateRaw = opState.getRawOperatorState(); + allParallelManagedOpStates.add(opStateBackend); + allParallelRawOpStates.add(opStateRaw); + if (idx == headOpIndex) { + Collection keyedStateBackend = opState.getManagedKeyedState(); + Collection keyGroupStateRaw = opState.getRawKeyedState(); + compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend); + compareKeyedState(Collections.singletonList(originalKeyedStateRaw), keyGroupStateRaw); + } } - compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend); - compareKeyedState(Collections.singletonList(originalKeyedStateRaw), keyGroupStateRaw); + actualOpStatesBackend.add(allParallelManagedOpStates); + actualOpStatesRaw.add(allParallelRawOpStates); } + comparePartitionableState(expectedOpStatesBackend, actualOpStatesBackend); comparePartitionableState(expectedOpStatesRaw, actualOpStatesRaw); } @@ -2578,14 +2566,11 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception operatorStates.put(id.f1, taskState); for (int index = 0; index < taskState.getParallelism(); index++) { StreamStateHandle subNonPartitionedState = - generateStateForVertex(id.f0, index) - .get(0); + generateStateForVertex(id.f0, index); OperatorStateHandle subManagedOperatorState = - generateChainedPartitionableStateHandle(id.f0, index, 2, 8, false) - .get(0); + generatePartitionableStateHandle(id.f0, index, 2, 8, false); OperatorStateHandle subRawOperatorState = - generateChainedPartitionableStateHandle(id.f0, index, 2, 8, true) - .get(0); + generatePartitionableStateHandle(id.f0, index, 2, 8, true); OperatorSubtaskState subtaskState = new OperatorSubtaskState(subNonPartitionedState, subManagedOperatorState, @@ -2707,57 +2692,75 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception for (int i = 0; i < newJobVertex1.getParallelism(); i++) { - TaskStateHandles taskStateHandles = newJobVertex1.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateHandles(); - ChainedStateHandle actualSubNonPartitionedState = taskStateHandles.getLegacyOperatorState(); - List> actualSubManagedOperatorState = taskStateHandles.getManagedOperatorState(); - List> actualSubRawOperatorState = taskStateHandles.getRawOperatorState(); + final List operatorIds = newJobVertex1.getOperatorIDs(); - assertNull(taskStateHandles.getManagedKeyedState()); - assertNull(taskStateHandles.getRawKeyedState()); + TaskStateSnapshot stateSnapshot = newJobVertex1.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateSnapshot(); + + OperatorSubtaskState headOpState = stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIds.size() - 1)); + assertTrue(headOpState.getManagedKeyedState().isEmpty()); + assertTrue(headOpState.getRawKeyedState().isEmpty()); // operator5 { int operatorIndexInChain = 2; - assertNull(actualSubNonPartitionedState.get(operatorIndexInChain)); - assertNull(actualSubManagedOperatorState.get(operatorIndexInChain)); - assertNull(actualSubRawOperatorState.get(operatorIndexInChain)); + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + + assertNull(opState.getLegacyOperatorState()); + assertTrue(opState.getManagedOperatorState().isEmpty()); + assertTrue(opState.getRawOperatorState().isEmpty()); } // operator1 { int operatorIndexInChain = 1; - ChainedStateHandle expectSubNonPartitionedState = generateStateForVertex(id1.f0, i); - ChainedStateHandle expectedManagedOpState = generateChainedPartitionableStateHandle( + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + + StreamStateHandle expectSubNonPartitionedState = generateStateForVertex(id1.f0, i); + OperatorStateHandle expectedManagedOpState = generatePartitionableStateHandle( id1.f0, i, 2, 8, false); - ChainedStateHandle expectedRawOpState = generateChainedPartitionableStateHandle( + OperatorStateHandle expectedRawOpState = generatePartitionableStateHandle( id1.f0, i, 2, 8, true); assertTrue(CommonTestUtils.isSteamContentEqual( - expectSubNonPartitionedState.get(0).openInputStream(), - actualSubNonPartitionedState.get(operatorIndexInChain).openInputStream())); - - assertTrue(CommonTestUtils.isSteamContentEqual(expectedManagedOpState.get(0).openInputStream(), - actualSubManagedOperatorState.get(operatorIndexInChain).iterator().next().openInputStream())); - - assertTrue(CommonTestUtils.isSteamContentEqual(expectedRawOpState.get(0).openInputStream(), - actualSubRawOperatorState.get(operatorIndexInChain).iterator().next().openInputStream())); + expectSubNonPartitionedState.openInputStream(), + opState.getLegacyOperatorState().openInputStream())); + + Collection managedOperatorState = opState.getManagedOperatorState(); + assertEquals(1, managedOperatorState.size()); + assertTrue(CommonTestUtils.isSteamContentEqual(expectedManagedOpState.openInputStream(), + managedOperatorState.iterator().next().openInputStream())); + + Collection rawOperatorState = opState.getRawOperatorState(); + assertEquals(1, rawOperatorState.size()); + assertTrue(CommonTestUtils.isSteamContentEqual(expectedRawOpState.openInputStream(), + rawOperatorState.iterator().next().openInputStream())); } // operator2 { int operatorIndexInChain = 0; - ChainedStateHandle expectSubNonPartitionedState = generateStateForVertex(id2.f0, i); - ChainedStateHandle expectedManagedOpState = generateChainedPartitionableStateHandle( + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + + StreamStateHandle expectSubNonPartitionedState = generateStateForVertex(id2.f0, i); + OperatorStateHandle expectedManagedOpState = generatePartitionableStateHandle( id2.f0, i, 2, 8, false); - ChainedStateHandle expectedRawOpState = generateChainedPartitionableStateHandle( + OperatorStateHandle expectedRawOpState = generatePartitionableStateHandle( id2.f0, i, 2, 8, true); - assertTrue(CommonTestUtils.isSteamContentEqual(expectSubNonPartitionedState.get(0).openInputStream(), - actualSubNonPartitionedState.get(operatorIndexInChain).openInputStream())); - - assertTrue(CommonTestUtils.isSteamContentEqual(expectedManagedOpState.get(0).openInputStream(), - actualSubManagedOperatorState.get(operatorIndexInChain).iterator().next().openInputStream())); - - assertTrue(CommonTestUtils.isSteamContentEqual(expectedRawOpState.get(0).openInputStream(), - actualSubRawOperatorState.get(operatorIndexInChain).iterator().next().openInputStream())); + assertTrue(CommonTestUtils.isSteamContentEqual( + expectSubNonPartitionedState.openInputStream(), + opState.getLegacyOperatorState().openInputStream())); + + Collection managedOperatorState = opState.getManagedOperatorState(); + assertEquals(1, managedOperatorState.size()); + assertTrue(CommonTestUtils.isSteamContentEqual(expectedManagedOpState.openInputStream(), + managedOperatorState.iterator().next().openInputStream())); + + Collection rawOperatorState = opState.getRawOperatorState(); + assertEquals(1, rawOperatorState.size()); + assertTrue(CommonTestUtils.isSteamContentEqual(expectedRawOpState.openInputStream(), + rawOperatorState.iterator().next().openInputStream())); } } @@ -2765,38 +2768,48 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception List>> actualRawOperatorStates = new ArrayList<>(newJobVertex2.getParallelism()); for (int i = 0; i < newJobVertex2.getParallelism(); i++) { - TaskStateHandles taskStateHandles = newJobVertex2.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateHandles(); + + final List operatorIds = newJobVertex2.getOperatorIDs(); + + TaskStateSnapshot stateSnapshot = newJobVertex2.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateSnapshot(); // operator 3 { int operatorIndexInChain = 1; + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + List> actualSubManagedOperatorState = new ArrayList<>(1); - actualSubManagedOperatorState.add(taskStateHandles.getManagedOperatorState().get(operatorIndexInChain)); + actualSubManagedOperatorState.add(opState.getManagedOperatorState()); List> actualSubRawOperatorState = new ArrayList<>(1); - actualSubRawOperatorState.add(taskStateHandles.getRawOperatorState().get(operatorIndexInChain)); + actualSubRawOperatorState.add(opState.getRawOperatorState()); actualManagedOperatorStates.add(actualSubManagedOperatorState); actualRawOperatorStates.add(actualSubRawOperatorState); - assertNull(taskStateHandles.getLegacyOperatorState().get(operatorIndexInChain)); + assertNull(opState.getLegacyOperatorState()); } // operator 6 { int operatorIndexInChain = 0; - assertNull(taskStateHandles.getManagedOperatorState().get(operatorIndexInChain)); - assertNull(taskStateHandles.getRawOperatorState().get(operatorIndexInChain)); - assertNull(taskStateHandles.getLegacyOperatorState().get(operatorIndexInChain)); + OperatorSubtaskState opState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIndexInChain)); + assertNull(opState.getLegacyOperatorState()); + assertTrue(opState.getManagedOperatorState().isEmpty()); + assertTrue(opState.getRawOperatorState().isEmpty()); } KeyGroupsStateHandle originalKeyedStateBackend = generateKeyGroupState(id3.f0, newKeyGroupPartitions2.get(i), false); KeyGroupsStateHandle originalKeyedStateRaw = generateKeyGroupState(id3.f0, newKeyGroupPartitions2.get(i), true); + OperatorSubtaskState headOpState = + stateSnapshot.getSubtaskStateByOperatorID(operatorIds.get(operatorIds.size() - 1)); - Collection keyedStateBackend = taskStateHandles.getManagedKeyedState(); - Collection keyGroupStateRaw = taskStateHandles.getRawKeyedState(); + Collection keyedStateBackend = headOpState.getManagedKeyedState(); + Collection keyGroupStateRaw = headOpState.getRawKeyedState(); compareKeyedState(Collections.singletonList(originalKeyedStateBackend), keyedStateBackend); @@ -2974,19 +2987,50 @@ public static Tuple2> serializeTogetherAndTrackOffsets( return new Tuple2<>(allSerializedValuesConcatenated, offsets); } - public static ChainedStateHandle generateStateForVertex( + public static StreamStateHandle generateStateForVertex( JobVertexID jobVertexID, int index) throws IOException { Random random = new Random(jobVertexID.hashCode() + index); int value = random.nextInt(); - return generateChainedStateHandle(value); + return generateStreamStateHandle(value); + } + + public static StreamStateHandle generateStreamStateHandle(Serializable value) throws IOException { + return TestByteStreamStateHandleDeepCompare.fromSerializable(String.valueOf(UUID.randomUUID()), value); } public static ChainedStateHandle generateChainedStateHandle( Serializable value) throws IOException { return ChainedStateHandle.wrapSingleHandle( - TestByteStreamStateHandleDeepCompare.fromSerializable(String.valueOf(UUID.randomUUID()), value)); + generateStreamStateHandle(value)); + } + + public static OperatorStateHandle generatePartitionableStateHandle( + JobVertexID jobVertexID, + int index, + int namedStates, + int partitionsPerState, + boolean rawState) throws IOException { + + Map> statesListsMap = new HashMap<>(namedStates); + + for (int i = 0; i < namedStates; ++i) { + List testStatesLists = new ArrayList<>(partitionsPerState); + // generate state + int seed = jobVertexID.hashCode() * index + i * namedStates; + if (rawState) { + seed = (seed + 1) * 31; + } + Random random = new Random(seed); + for (int j = 0; j < partitionsPerState; ++j) { + int simulatedStateValue = random.nextInt(); + testStatesLists.add(simulatedStateValue); + } + statesListsMap.put("state-" + i, testStatesLists); + } + + return generatePartitionableStateHandle(statesListsMap); } public static ChainedStateHandle generateChainedPartitionableStateHandle( @@ -3013,11 +3057,11 @@ public static ChainedStateHandle generateChainedPartitionab statesListsMap.put("state-" + i, testStatesLists); } - return generateChainedPartitionableStateHandle(statesListsMap); + return ChainedStateHandle.wrapSingleHandle(generatePartitionableStateHandle(statesListsMap)); } - private static ChainedStateHandle generateChainedPartitionableStateHandle( - Map> states) throws IOException { + private static OperatorStateHandle generatePartitionableStateHandle( + Map> states) throws IOException { List> namedStateSerializables = new ArrayList<>(states.size()); @@ -3032,20 +3076,18 @@ private static ChainedStateHandle generateChainedPartitiona int idx = 0; for (Map.Entry> entry : states.entrySet()) { offsetsMap.put( - entry.getKey(), - new OperatorStateHandle.StateMetaInfo( - serializationWithOffsets.f1.get(idx), - OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); + entry.getKey(), + new OperatorStateHandle.StateMetaInfo( + serializationWithOffsets.f1.get(idx), + OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); ++idx; } ByteStreamStateHandle streamStateHandle = new TestByteStreamStateHandleDeepCompare( - String.valueOf(UUID.randomUUID()), - serializationWithOffsets.f0); + String.valueOf(UUID.randomUUID()), + serializationWithOffsets.f0); - OperatorStateHandle operatorStateHandle = - new OperatorStateHandle(offsetsMap, streamStateHandle); - return ChainedStateHandle.wrapSingleHandle(operatorStateHandle); + return new OperatorStateHandle(offsetsMap, streamStateHandle); } static ExecutionJobVertex mockExecutionJobVertex( @@ -3139,24 +3181,23 @@ private static ExecutionVertex mockExecutionVertex( return vertex; } - static SubtaskState mockSubtaskState( + static TaskStateSnapshot mockSubtaskState( JobVertexID jobVertexID, int index, KeyGroupRange keyGroupRange) throws IOException { - ChainedStateHandle nonPartitionedState = generateStateForVertex(jobVertexID, index); - ChainedStateHandle partitionableState = generateChainedPartitionableStateHandle(jobVertexID, index, 2, 8, false); + StreamStateHandle nonPartitionedState = generateStateForVertex(jobVertexID, index); + OperatorStateHandle partitionableState = generatePartitionableStateHandle(jobVertexID, index, 2, 8, false); KeyGroupsStateHandle partitionedKeyGroupState = generateKeyGroupState(jobVertexID, keyGroupRange, false); - SubtaskState subtaskState = mock(SubtaskState.class, withSettings().serializable()); + TaskStateSnapshot subtaskStates = spy(new TaskStateSnapshot()); + OperatorSubtaskState subtaskState = spy(new OperatorSubtaskState( + nonPartitionedState, partitionableState, null, partitionedKeyGroupState, null) + ); - doReturn(nonPartitionedState).when(subtaskState).getLegacyOperatorState(); - doReturn(partitionableState).when(subtaskState).getManagedOperatorState(); - doReturn(null).when(subtaskState).getRawOperatorState(); - doReturn(partitionedKeyGroupState).when(subtaskState).getManagedKeyedState(); - doReturn(null).when(subtaskState).getRawKeyedState(); + subtaskStates.putSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID), subtaskState); - return subtaskState; + return subtaskStates; } public static void verifyStateRestore( @@ -3165,27 +3206,27 @@ public static void verifyStateRestore( for (int i = 0; i < executionJobVertex.getParallelism(); i++) { - TaskStateHandles taskStateHandles = executionJobVertex.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateHandles(); + final List operatorIds = executionJobVertex.getOperatorIDs(); - ChainedStateHandle expectNonPartitionedState = generateStateForVertex(jobVertexID, i); - ChainedStateHandle actualNonPartitionedState = taskStateHandles.getLegacyOperatorState(); + TaskStateSnapshot stateSnapshot = executionJobVertex.getTaskVertices()[i].getCurrentExecutionAttempt().getTaskStateSnapshot(); + + OperatorSubtaskState operatorState = stateSnapshot.getSubtaskStateByOperatorID(OperatorID.fromJobVertexID(jobVertexID)); + + StreamStateHandle expectNonPartitionedState = generateStateForVertex(jobVertexID, i); assertTrue(CommonTestUtils.isSteamContentEqual( - expectNonPartitionedState.get(0).openInputStream(), - actualNonPartitionedState.get(0).openInputStream())); + expectNonPartitionedState.openInputStream(), + operatorState.getLegacyOperatorState().openInputStream())); ChainedStateHandle expectedOpStateBackend = generateChainedPartitionableStateHandle(jobVertexID, i, 2, 8, false); - List> actualPartitionableState = taskStateHandles.getManagedOperatorState(); - assertTrue(CommonTestUtils.isSteamContentEqual( expectedOpStateBackend.get(0).openInputStream(), - actualPartitionableState.get(0).iterator().next().openInputStream())); + operatorState.getManagedOperatorState().iterator().next().openInputStream())); KeyGroupsStateHandle expectPartitionedKeyGroupState = generateKeyGroupState( jobVertexID, keyGroupPartitions.get(i), false); - Collection actualPartitionedKeyGroupState = taskStateHandles.getManagedKeyedState(); - compareKeyedState(Collections.singletonList(expectPartitionedKeyGroupState), actualPartitionedKeyGroupState); + compareKeyedState(Collections.singletonList(expectPartitionedKeyGroupState), operatorState.getManagedKeyedState()); } } @@ -3632,17 +3673,4 @@ public void testSavepointsAreNotAddedToCompletedCheckpointStore() throws Excepti "The latest completed (proper) checkpoint should have been added to the completed checkpoint store.", completedCheckpointStore.getLatestCheckpoint().getCheckpointID() == checkpointIDCounter.getLast()); } - - private static final class SpyInjectingOperatorState extends OperatorState { - - private static final long serialVersionUID = -4004437428483663815L; - - public SpyInjectingOperatorState(OperatorID taskID, int parallelism, int maxParallelism) { - super(taskID, parallelism, maxParallelism); - } - - public void putState(int subtaskIndex, OperatorSubtaskState subtaskState) { - super.putState(subtaskIndex, spy(subtaskState)); - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index 7d2456881fc33..6ce071b2269bd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -34,18 +34,18 @@ import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.util.SerializableObject; + import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.junit.Test; import org.mockito.Mockito; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -118,10 +118,20 @@ public void testSetState() { PendingCheckpoint pending = coord.getPendingCheckpoints().values().iterator().next(); final long checkpointId = pending.getCheckpointId(); - SubtaskState checkpointStateHandles = new SubtaskState(serializedState, null, null, serializedKeyGroupStates, null); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), checkpointStateHandles)); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec2.getAttemptId(), checkpointId, new CheckpointMetrics(), checkpointStateHandles)); - coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec3.getAttemptId(), checkpointId, new CheckpointMetrics(), checkpointStateHandles)); + final TaskStateSnapshot subtaskStates = new TaskStateSnapshot(); + + subtaskStates.putSubtaskStateByOperatorID( + OperatorID.fromJobVertexID(statefulId), + new OperatorSubtaskState( + serializedState.get(0), + Collections.emptyList(), + Collections.emptyList(), + Collections.singletonList(serializedKeyGroupStates), + Collections.emptyList())); + + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec1.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStates)); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec2.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStates)); + coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statefulExec3.getAttemptId(), checkpointId, new CheckpointMetrics(), subtaskStates)); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec1.getAttemptId(), checkpointId)); coord.receiveAcknowledgeMessage(new AcknowledgeCheckpoint(jid, statelessExec2.getAttemptId(), checkpointId)); @@ -133,33 +143,26 @@ public void testSetState() { // verify that each stateful vertex got the state - final TaskStateHandles taskStateHandles = new TaskStateHandles( - serializedState, - Collections.>singletonList(null), - Collections.>singletonList(null), - Collections.singletonList(serializedKeyGroupStates), - null); - - BaseMatcher matcher = new BaseMatcher() { + BaseMatcher matcher = new BaseMatcher() { @Override public boolean matches(Object o) { - if (o instanceof TaskStateHandles) { - return o.equals(taskStateHandles); + if (o instanceof TaskStateSnapshot) { + return Objects.equals(o, subtaskStates); } return false; } @Override public void describeTo(Description description) { - description.appendValue(taskStateHandles); + description.appendValue(subtaskStates); } }; verify(statefulExec1, times(1)).setInitialState(Mockito.argThat(matcher)); verify(statefulExec2, times(1)).setInitialState(Mockito.argThat(matcher)); verify(statefulExec3, times(1)).setInitialState(Mockito.argThat(matcher)); - verify(statelessExec1, times(0)).setInitialState(Mockito.any()); - verify(statelessExec2, times(0)).setInitialState(Mockito.any()); + verify(statelessExec1, times(0)).setInitialState(Mockito.any()); + verify(statelessExec2, times(0)).setInitialState(Mockito.any()); } catch (Exception e) { e.printStackTrace(); @@ -250,9 +253,9 @@ public void testNonRestoredState() throws Exception { Map checkpointTaskStates = new HashMap<>(); { OperatorState taskState = new OperatorState(operatorId1, 3, 3); - taskState.putState(0, new OperatorSubtaskState(serializedState, null, null, null, null)); - taskState.putState(1, new OperatorSubtaskState(serializedState, null, null, null, null)); - taskState.putState(2, new OperatorSubtaskState(serializedState, null, null, null, null)); + taskState.putState(0, new OperatorSubtaskState(serializedState)); + taskState.putState(1, new OperatorSubtaskState(serializedState)); + taskState.putState(2, new OperatorSubtaskState(serializedState)); checkpointTaskStates.put(operatorId1, taskState); } @@ -279,7 +282,7 @@ public void testNonRestoredState() throws Exception { // There is no task for this { OperatorState taskState = new OperatorState(newOperatorID, 1, 1); - taskState.putState(0, new OperatorSubtaskState(serializedState, null, null, null, null)); + taskState.putState(0, new OperatorSubtaskState(serializedState)); checkpointTaskStates.put(newOperatorID, taskState); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java index 1fe4e65979cc8..320dc2df52bfb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreTest.java @@ -331,7 +331,7 @@ static class TestOperatorSubtaskState extends OperatorSubtaskState { boolean discarded; public TestOperatorSubtaskState() { - super(null, null, null, null, null); + super(); this.registered = false; this.discarded = false; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index 7d103d0b297f0..7ebb49a4c1c61 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -324,7 +324,7 @@ public void testNullSubtaskStateLeadsToStatelessTask() throws Exception { @Test public void testNonNullSubtaskStateLeadsToStatefulTask() throws Exception { PendingCheckpoint pending = createPendingCheckpoint(CheckpointProperties.forStandardCheckpoint(), null); - pending.acknowledgeTask(ATTEMPT_ID, mock(SubtaskState.class), mock(CheckpointMetrics.class)); + pending.acknowledgeTask(ATTEMPT_ID, mock(TaskStateSnapshot.class), mock(CheckpointMetrics.class)); Assert.assertFalse(pending.getOperatorStates().isEmpty()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java index 36c9cadeaec3e..9ed4851cad516 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.JobInformation; @@ -30,7 +31,6 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.operators.BatchTask; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.util.SerializedValue; import org.junit.Test; @@ -73,7 +73,7 @@ public void testSerialization() { final SerializedValue serializedJobVertexInformation = new SerializedValue<>(new TaskInformation( vertexID, taskName, currentNumberOfSubtasks, numberOfKeyGroups, invokableClass.getName(), taskConfiguration)); final int targetSlotNumber = 47; - final TaskStateHandles taskStateHandles = new TaskStateHandles(); + final TaskStateSnapshot taskStateHandles = new TaskStateSnapshot(); final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor( serializedJobInformation, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java index 0eed90d271bcc..c9b7a40a78b0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexLocalityTest.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; @@ -38,7 +39,6 @@ import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; import org.apache.flink.runtime.jobmanager.slots.SlotOwner; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; @@ -51,8 +51,10 @@ import java.util.Iterator; import java.util.concurrent.TimeUnit; -import static org.mockito.Mockito.*; -import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; /** * Tests that the execution vertex handles locality preferences well. @@ -169,7 +171,7 @@ public void testLocalityBasedOnState() throws Exception { // target state ExecutionVertex target = graph.getAllVertices().get(targetVertexId).getTaskVertices()[i]; - target.getCurrentExecutionAttempt().setInitialState(mock(TaskStateHandles.class)); + target.getCurrentExecutionAttempt().setInitialState(mock(TaskStateSnapshot.class)); } // validate that the target vertices have the state's location as the location preference diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java index a63b02d785f19..23f0a389076c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java @@ -18,16 +18,6 @@ package org.apache.flink.runtime.jobmanager; -import akka.actor.ActorRef; -import akka.actor.ActorSystem; -import akka.actor.Identify; -import akka.actor.PoisonPill; -import akka.actor.Props; -import akka.japi.pf.FI; -import akka.japi.pf.ReceiveBuilder; -import akka.pattern.Patterns; -import akka.testkit.CallingThreadDispatcher; -import akka.testkit.JavaTestKit; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -44,8 +34,9 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.executiongraph.restart.FixedDelayRestartStrategy; @@ -59,6 +50,7 @@ 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.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; @@ -69,9 +61,6 @@ import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.messages.JobManagerMessages; import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.state.ChainedStateHandle; -import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.taskmanager.TaskManager; import org.apache.flink.runtime.testingUtils.TestingJobManager; @@ -83,23 +72,24 @@ import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare; import org.apache.flink.util.InstantiationUtil; - import org.apache.flink.util.TestLogger; + +import akka.actor.ActorRef; +import akka.actor.ActorSystem; +import akka.actor.Identify; +import akka.actor.PoisonPill; +import akka.actor.Props; +import akka.japi.pf.FI; +import akka.japi.pf.ReceiveBuilder; +import akka.pattern.Patterns; +import akka.testkit.CallingThreadDispatcher; +import akka.testkit.JavaTestKit; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import scala.Int; -import scala.Option; -import scala.PartialFunction; -import scala.concurrent.Await; -import scala.concurrent.Future; -import scala.concurrent.duration.Deadline; -import scala.concurrent.duration.FiniteDuration; -import scala.runtime.BoxedUnit; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -113,6 +103,15 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import scala.Int; +import scala.Option; +import scala.PartialFunction; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.Deadline; +import scala.concurrent.duration.FiniteDuration; +import scala.runtime.BoxedUnit; + import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -552,10 +551,10 @@ public static class BlockingStatefulInvokable extends BlockingInvokable implemen @Override public void setInitialState( - TaskStateHandles taskStateHandles) throws Exception { + TaskStateSnapshot taskStateHandles) throws Exception { int subtaskIndex = getIndexInSubtaskGroup(); if (subtaskIndex < recoveredStates.length) { - try (FSDataInputStream in = taskStateHandles.getLegacyOperatorState().get(0).openInputStream()) { + try (FSDataInputStream in = taskStateHandles.getSubtaskStateMappings().iterator().next().getValue().getLegacyOperatorState().openInputStream()) { recoveredStates[subtaskIndex] = InstantiationUtil.deserializeObject(in, getUserCodeClassLoader()); } } @@ -567,10 +566,11 @@ public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, Checkpoi String.valueOf(UUID.randomUUID()), InstantiationUtil.serializeObject(checkpointMetaData.getCheckpointId())); - ChainedStateHandle chainedStateHandle = - new ChainedStateHandle(Collections.singletonList(byteStreamStateHandle)); - SubtaskState checkpointStateHandles = - new SubtaskState(chainedStateHandle, null, null, null, null); + TaskStateSnapshot checkpointStateHandles = new TaskStateSnapshot(); + checkpointStateHandles.putSubtaskStateByOperatorID( + OperatorID.fromJobVertexID(getEnvironment().getJobVertexId()), + new OperatorSubtaskState(byteStreamStateHandle) + ); getEnvironment().acknowledgeCheckpoint( checkpointMetaData.getCheckpointId(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java index bc420cc27799b..d022cdcf59e69 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/CheckpointMessagesTest.java @@ -24,14 +24,17 @@ import org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTest; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.NotifyCheckpointComplete; import org.apache.flink.runtime.messages.checkpoint.TriggerCheckpoint; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.StreamStateHandle; + import org.junit.Test; import java.io.IOException; @@ -68,13 +71,17 @@ public void testConfirmTaskCheckpointed() { KeyGroupRange keyGroupRange = KeyGroupRange.of(42,42); - SubtaskState checkpointStateHandles = - new SubtaskState( - CheckpointCoordinatorTest.generateChainedStateHandle(new MyHandle()), - CheckpointCoordinatorTest.generateChainedPartitionableStateHandle(new JobVertexID(), 0, 2, 8, false), - null, - CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, Collections.singletonList(new MyHandle())), - null); + TaskStateSnapshot checkpointStateHandles = new TaskStateSnapshot(); + checkpointStateHandles.putSubtaskStateByOperatorID( + new OperatorID(), + new OperatorSubtaskState( + CheckpointCoordinatorTest.generateStreamStateHandle(new MyHandle()), + CheckpointCoordinatorTest.generatePartitionableStateHandle(new JobVertexID(), 0, 2, 8, false), + null, + CheckpointCoordinatorTest.generateKeyGroupState(keyGroupRange, Collections.singletonList(new MyHandle())), + null + ) + ); AcknowledgeCheckpoint withState = new AcknowledgeCheckpoint( new JobID(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java index 851fa967be729..8ed06b2ef3682 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DummyEnvironment.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -156,7 +156,7 @@ public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpoin } @Override - public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, SubtaskState subtaskState) { + public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, TaskStateSnapshot subtaskState) { } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java index 4f0242e131ab5..7514cc4200d74 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -50,8 +50,8 @@ import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.types.Record; import org.apache.flink.util.MutableObjectIterator; - import org.apache.flink.util.Preconditions; + import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -354,7 +354,7 @@ public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpoin } @Override - public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, SubtaskState subtaskState) { + public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, TaskStateSnapshot subtaskState) { throw new UnsupportedOperationException(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index c6d2fec2f0daf..085a38699772f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -27,6 +27,7 @@ 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.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; @@ -49,7 +50,6 @@ import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.util.SerializedValue; @@ -187,7 +187,7 @@ private static Task createTask() throws Exception { Collections.emptyList(), Collections.emptyList(), 0, - new TaskStateHandles(), + new TaskStateSnapshot(), mock(MemoryManager.class), mock(IOManager.class), networkEnvironment, @@ -228,7 +228,7 @@ public void invoke() throws Exception { } @Override - public void setInitialState(TaskStateHandles taskStateHandles) throws Exception {} + public void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception {} @Override public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index 40678de125424..1ebd4adf73883 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -20,39 +20,41 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.executiongraph.JobInformation; -import org.apache.flink.runtime.executiongraph.TaskInformation; -import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker; -import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; -import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; -import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; -import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; -import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.JobInformation; +import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker; +import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; +import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.jobgraph.tasks.StoppableTask; import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.state.TaskStateHandles; +import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; +import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; + import org.junit.Test; import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import scala.concurrent.duration.FiniteDuration; import java.lang.reflect.Field; import java.util.Collections; import java.util.concurrent.Executor; +import scala.concurrent.duration.FiniteDuration; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -88,7 +90,7 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { Collections.emptyList(), Collections.emptyList(), 0, - mock(TaskStateHandles.class), + mock(TaskStateSnapshot.class), mock(MemoryManager.class), mock(IOManager.class), mock(NetworkEnvironment.class), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java index f262bf2c0b631..c1df5a376d492 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; @@ -70,7 +70,8 @@ import java.util.concurrent.Executors; import static org.junit.Assume.assumeTrue; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Test that verifies the behavior of blocking shutdown hooks and of the @@ -232,7 +233,7 @@ public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) { private static final class NoOpCheckpointResponder implements CheckpointResponder { @Override - public void acknowledgeCheckpoint(JobID j, ExecutionAttemptID e, long i, CheckpointMetrics c, SubtaskState s) {} + public void acknowledgeCheckpoint(JobID j, ExecutionAttemptID e, long i, CheckpointMetrics c, TaskStateSnapshot s) {} @Override public void declineCheckpoint(JobID j, ExecutionAttemptID e, long l, Throwable t) {} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java index 77caa34d11fbf..13100db01d837 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.util.CorruptConfigurationException; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.util.ClassLoaderUtil; @@ -76,6 +77,7 @@ public class StreamConfig implements Serializable { private static final String OUT_STREAM_EDGES = "outStreamEdges"; private static final String IN_STREAM_EDGES = "inStreamEdges"; private static final String OPERATOR_NAME = "operatorName"; + private static final String OPERATOR_ID = "operatorID"; private static final String CHAIN_END = "chainEnd"; private static final String CHECKPOINTING_ENABLED = "checkpointing"; @@ -213,7 +215,7 @@ public void setStreamOperator(StreamOperator operator) { } } - public T getStreamOperator(ClassLoader cl) { + public > T getStreamOperator(ClassLoader cl) { try { return InstantiationUtil.readObjectFromConfig(this.config, SERIALIZEDUDF, cl); } @@ -411,6 +413,15 @@ public Map getTransitiveChainedTaskConfigs(ClassLoader cl } } + public void setOperatorID(OperatorID operatorID) { + this.config.setBytes(OPERATOR_ID, operatorID.getBytes()); + } + + public OperatorID getOperatorID() { + byte[] operatorIDBytes = config.getBytes(OPERATOR_ID, null); + return new OperatorID(Preconditions.checkNotNull(operatorIDBytes)); + } + public void setOperatorName(String name) { this.config.setString(OPERATOR_NAME, name); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java index e70962b9b2623..abaa74e7d2ecc 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java @@ -246,7 +246,9 @@ private List createChain( operatorHashes = new ArrayList<>(); chainedOperatorHashes.put(startNodeId, operatorHashes); } - operatorHashes.add(new Tuple2<>(hashes.get(currentNodeId), legacyHashes.get(1).get(currentNodeId))); + + byte[] primaryHashBytes = hashes.get(currentNodeId); + operatorHashes.add(new Tuple2<>(primaryHashBytes, legacyHashes.get(1).get(currentNodeId))); chainedNames.put(currentNodeId, createChainedName(currentNodeId, chainableOutputs)); chainedMinResources.put(currentNodeId, createChainedMinResources(currentNodeId, chainableOutputs)); @@ -280,13 +282,16 @@ private List createChain( chainedConfigs.put(startNodeId, new HashMap()); } config.setChainIndex(chainIndex); - config.setOperatorName(streamGraph.getStreamNode(currentNodeId).getOperatorName()); + StreamNode node = streamGraph.getStreamNode(currentNodeId); + config.setOperatorName(node.getOperatorName()); chainedConfigs.get(startNodeId).put(currentNodeId, config); } + + config.setOperatorID(new OperatorID(primaryHashBytes)); + if (chainableOutputs.isEmpty()) { config.setChainEnd(); } - return transitiveOutEdges; } else { 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 d711518e49f4c..a72b9fe3491e7 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 @@ -36,6 +36,8 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.CheckpointOptions.CheckpointType; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.CheckpointStreamFactory; @@ -60,7 +62,6 @@ import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.util.OutputTag; @@ -179,7 +180,6 @@ public abstract class AbstractStreamOperator public void setup(StreamTask containingTask, StreamConfig config, Output> output) { this.container = containingTask; this.config = config; - this.metrics = container.getEnvironment().getMetricGroup().addOperator(config.getOperatorName()); this.output = new CountingOutput(output, ((OperatorMetricGroup) this.metrics).getIOMetricGroup().getNumRecordsOutCounter()); if (config.isChainStart()) { @@ -208,13 +208,13 @@ public MetricGroup getMetricGroup() { } @Override - public final void initializeState(OperatorStateHandles stateHandles) throws Exception { + public final void initializeState(OperatorSubtaskState stateHandles) throws Exception { Collection keyedStateHandlesRaw = null; Collection operatorStateHandlesRaw = null; Collection operatorStateHandlesBackend = null; - boolean restoring = null != stateHandles; + boolean restoring = (null != stateHandles); initKeyedState(); //TODO we should move the actual initialization of this from StreamTask to this class @@ -266,13 +266,13 @@ public final void initializeState(OperatorStateHandles stateHandles) throws Exce * Can be removed when we remove the APIs for non-repartitionable operator state. */ @Deprecated - private void restoreStreamCheckpointed(OperatorStateHandles stateHandles) throws Exception { + private void restoreStreamCheckpointed(OperatorSubtaskState stateHandles) throws Exception { StreamStateHandle state = stateHandles.getLegacyOperatorState(); if (null != state) { if (this instanceof CheckpointedRestoringOperator) { - LOG.debug("Restore state of task {} in chain ({}).", - stateHandles.getOperatorChainIndex(), getContainingTask().getName()); + LOG.debug("Restore state of task {} in operator with id ({}).", + getContainingTask().getName(), getOperatorID()); FSDataInputStream is = state.openInputStream(); try { @@ -973,6 +973,11 @@ public void processWatermark2(Watermark mark) throws Exception { } } + @Override + public OperatorID getOperatorID() { + return config.getOperatorID(); + } + @VisibleForTesting public int numProcessingTimeTimers() { return timeServiceManager == null ? 0 : diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java index 61578b23a6d51..9d5e02b1a3d37 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java @@ -20,10 +20,11 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; import org.apache.flink.streaming.runtime.tasks.StreamTask; import java.io.Serializable; @@ -123,7 +124,7 @@ StreamStateHandle snapshotLegacyOperatorState( * * @param stateHandles state handles to the operator state. */ - void initializeState(OperatorStateHandles stateHandles) throws Exception; + void initializeState(OperatorSubtaskState stateHandles) throws Exception; /** * Called when the checkpoint with the given ID is completed and acknowledged on the JobManager. @@ -149,4 +150,5 @@ StreamStateHandle snapshotLegacyOperatorState( MetricGroup getMetricGroup(); + OperatorID getOperatorID(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java index 1a79f5429c206..4914075d0136d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorStateHandles.java @@ -20,13 +20,10 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.util.CollectionUtil; -import org.apache.flink.util.Preconditions; import java.util.Collection; import java.util.List; @@ -63,22 +60,6 @@ public OperatorStateHandles( this.rawOperatorState = rawOperatorState; } - public OperatorStateHandles(TaskStateHandles taskStateHandles, int operatorChainIndex) { - Preconditions.checkNotNull(taskStateHandles); - - this.operatorChainIndex = operatorChainIndex; - - ChainedStateHandle legacyState = taskStateHandles.getLegacyOperatorState(); - this.legacyOperatorState = ChainedStateHandle.isNullOrEmpty(legacyState) ? - null : legacyState.get(operatorChainIndex); - - this.rawKeyedState = taskStateHandles.getRawKeyedState(); - this.managedKeyedState = taskStateHandles.getManagedKeyedState(); - - this.managedOperatorState = getSafeItemAtIndexOrNull(taskStateHandles.getManagedOperatorState(), operatorChainIndex); - this.rawOperatorState = getSafeItemAtIndexOrNull(taskStateHandles.getRawOperatorState(), operatorChainIndex); - } - public StreamStateHandle getLegacyOperatorState() { return legacyOperatorState; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index c35a6dc5b3684..cb8639b0dbe60 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -25,17 +25,18 @@ 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.SubtaskState; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.AbstractStateBackend; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; @@ -44,7 +45,6 @@ import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.StateUtil; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -54,7 +54,6 @@ import org.apache.flink.streaming.runtime.io.RecordWriterOutput; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.streamstatus.StreamStatusMaintainer; -import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FutureUtil; import org.apache.flink.util.Preconditions; @@ -64,13 +63,11 @@ import java.io.Closeable; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; -import java.util.List; +import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.RunnableFuture; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicReference; @@ -158,7 +155,7 @@ public abstract class StreamTask> /** The map of user-defined accumulators of this task. */ private Map> accumulatorMap; - private TaskStateHandles restoreStateHandles; + private TaskStateSnapshot taskStateSnapshot; /** The currently active background materialization threads. */ private final CloseableRegistry cancelables = new CloseableRegistry(); @@ -508,8 +505,8 @@ RecordWriterOutput[] getStreamOutputs() { // ------------------------------------------------------------------------ @Override - public void setInitialState(TaskStateHandles taskStateHandles) { - this.restoreStateHandles = taskStateHandles; + public void setInitialState(TaskStateSnapshot taskStateHandles) { + this.taskStateSnapshot = taskStateHandles; } @Override @@ -658,12 +655,11 @@ private void checkpointState( private void initializeState() throws Exception { - boolean restored = null != restoreStateHandles; + boolean restored = null != taskStateSnapshot; if (restored) { - checkRestorePreconditions(operatorChain.getChainLength()); initializeOperators(true); - restoreStateHandles = null; // free for GC + taskStateSnapshot = null; // free for GC } else { initializeOperators(false); } @@ -674,8 +670,8 @@ private void initializeOperators(boolean restored) throws Exception { for (int chainIdx = 0; chainIdx < allOperators.length; ++chainIdx) { StreamOperator operator = allOperators[chainIdx]; if (null != operator) { - if (restored && restoreStateHandles != null) { - operator.initializeState(new OperatorStateHandles(restoreStateHandles, chainIdx)); + if (restored && taskStateSnapshot != null) { + operator.initializeState(taskStateSnapshot.getSubtaskStateByOperatorID(operator.getOperatorID())); } else { operator.initializeState(null); } @@ -683,26 +679,6 @@ private void initializeOperators(boolean restored) throws Exception { } } - private void checkRestorePreconditions(int operatorChainLength) { - - ChainedStateHandle nonPartitionableOperatorStates = - restoreStateHandles.getLegacyOperatorState(); - List> operatorStates = - restoreStateHandles.getManagedOperatorState(); - - if (nonPartitionableOperatorStates != null) { - Preconditions.checkState(nonPartitionableOperatorStates.getLength() == operatorChainLength, - "Invalid Invalid number of operator states. Found :" + nonPartitionableOperatorStates.getLength() - + ". Expected: " + operatorChainLength); - } - - if (!CollectionUtil.isNullOrEmpty(operatorStates)) { - Preconditions.checkArgument(operatorStates.size() == operatorChainLength, - "Invalid number of operator states. Found :" + operatorStates.size() + - ". Expected: " + operatorChainLength); - } - } - // ------------------------------------------------------------------------ // State backend // ------------------------------------------------------------------------ @@ -768,8 +744,13 @@ public AbstractKeyedStateBackend createKeyedStateBackend( cancelables.registerClosable(keyedStateBackend); // restore if we have some old state - Collection restoreKeyedStateHandles = - restoreStateHandles == null ? null : restoreStateHandles.getManagedKeyedState(); + Collection restoreKeyedStateHandles = null; + + if (taskStateSnapshot != null) { + OperatorSubtaskState stateByOperatorID = + taskStateSnapshot.getSubtaskStateByOperatorID(headOperator.getOperatorID()); + restoreKeyedStateHandles = stateByOperatorID != null ? stateByOperatorID.getManagedKeyedState() : null; + } keyedStateBackend.restore(restoreKeyedStateHandles); @@ -850,12 +831,9 @@ private static final class AsyncCheckpointRunnable implements Runnable, Closeabl private final StreamTask owner; - private final List snapshotInProgressList; - - private RunnableFuture futureKeyedBackendStateHandles; - private RunnableFuture futureKeyedStreamStateHandles; + private final Map operatorSnapshotsInProgress; - private List nonPartitionedStateHandles; + private Map nonPartitionedStateHandles; private final CheckpointMetaData checkpointMetaData; private final CheckpointMetrics checkpointMetrics; @@ -867,81 +845,60 @@ private static final class AsyncCheckpointRunnable implements Runnable, Closeabl AsyncCheckpointRunnable( StreamTask owner, - List nonPartitionedStateHandles, - List snapshotInProgressList, + Map nonPartitionedStateHandles, + Map operatorSnapshotsInProgress, CheckpointMetaData checkpointMetaData, CheckpointMetrics checkpointMetrics, long asyncStartNanos) { this.owner = Preconditions.checkNotNull(owner); - this.snapshotInProgressList = Preconditions.checkNotNull(snapshotInProgressList); + this.operatorSnapshotsInProgress = Preconditions.checkNotNull(operatorSnapshotsInProgress); this.checkpointMetaData = Preconditions.checkNotNull(checkpointMetaData); this.checkpointMetrics = Preconditions.checkNotNull(checkpointMetrics); this.nonPartitionedStateHandles = nonPartitionedStateHandles; this.asyncStartNanos = asyncStartNanos; - - if (!snapshotInProgressList.isEmpty()) { - // TODO Currently only the head operator of a chain can have keyed state, so simply access it directly. - int headIndex = snapshotInProgressList.size() - 1; - OperatorSnapshotResult snapshotInProgress = snapshotInProgressList.get(headIndex); - if (null != snapshotInProgress) { - this.futureKeyedBackendStateHandles = snapshotInProgress.getKeyedStateManagedFuture(); - this.futureKeyedStreamStateHandles = snapshotInProgress.getKeyedStateRawFuture(); - } - } } @Override public void run() { FileSystemSafetyNet.initializeSafetyNetForThread(); try { - // Keyed state handle future, currently only one (the head) operator can have this - KeyedStateHandle keyedStateHandleBackend = FutureUtil.runIfNotDoneAndGet(futureKeyedBackendStateHandles); - KeyedStateHandle keyedStateHandleStream = FutureUtil.runIfNotDoneAndGet(futureKeyedStreamStateHandles); - - List operatorStatesBackend = new ArrayList<>(snapshotInProgressList.size()); - List operatorStatesStream = new ArrayList<>(snapshotInProgressList.size()); - - for (OperatorSnapshotResult snapshotInProgress : snapshotInProgressList) { - if (null != snapshotInProgress) { - operatorStatesBackend.add( - FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getOperatorStateManagedFuture())); - operatorStatesStream.add( - FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getOperatorStateRawFuture())); - } else { - operatorStatesBackend.add(null); - operatorStatesStream.add(null); - } - } + boolean hasState = false; + final TaskStateSnapshot taskOperatorSubtaskStates = + new TaskStateSnapshot(operatorSnapshotsInProgress.size()); - final long asyncEndNanos = System.nanoTime(); - final long asyncDurationMillis = (asyncEndNanos - asyncStartNanos) / 1_000_000; + for (Map.Entry entry : operatorSnapshotsInProgress.entrySet()) { - checkpointMetrics.setAsyncDurationMillis(asyncDurationMillis); + OperatorID operatorID = entry.getKey(); + OperatorSnapshotResult snapshotInProgress = entry.getValue(); - ChainedStateHandle chainedNonPartitionedOperatorsState = - new ChainedStateHandle<>(nonPartitionedStateHandles); + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( + nonPartitionedStateHandles.get(operatorID), + FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getOperatorStateManagedFuture()), + FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getOperatorStateRawFuture()), + FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getKeyedStateManagedFuture()), + FutureUtil.runIfNotDoneAndGet(snapshotInProgress.getKeyedStateRawFuture()) + ); - ChainedStateHandle chainedOperatorStateBackend = - new ChainedStateHandle<>(operatorStatesBackend); + hasState |= operatorSubtaskState.hasState(); + taskOperatorSubtaskStates.putSubtaskStateByOperatorID(operatorID, operatorSubtaskState); + } - ChainedStateHandle chainedOperatorStateStream = - new ChainedStateHandle<>(operatorStatesStream); + final long asyncEndNanos = System.nanoTime(); + final long asyncDurationMillis = (asyncEndNanos - asyncStartNanos) / 1_000_000; - SubtaskState subtaskState = createSubtaskStateFromSnapshotStateHandles( - chainedNonPartitionedOperatorsState, - chainedOperatorStateBackend, - chainedOperatorStateStream, - keyedStateHandleBackend, - keyedStateHandleStream); + checkpointMetrics.setAsyncDurationMillis(asyncDurationMillis); if (asyncCheckpointState.compareAndSet(CheckpointingOperation.AsynCheckpointState.RUNNING, CheckpointingOperation.AsynCheckpointState.COMPLETED)) { + // we signal stateless tasks by reporting null, so that there are no attempts to assign empty state + // to stateless tasks on restore. This enables simple job modifications that only concern + // stateless without the need to assign them uids to match their (always empty) states. owner.getEnvironment().acknowledgeCheckpoint( checkpointMetaData.getCheckpointId(), checkpointMetrics, - subtaskState); + hasState ? taskOperatorSubtaskStates : null); if (LOG.isDebugEnabled()) { LOG.debug("{} - finished asynchronous part of checkpoint {}. Asynchronous duration: {} ms", @@ -988,38 +945,13 @@ public void close() { } } - private SubtaskState createSubtaskStateFromSnapshotStateHandles( - ChainedStateHandle chainedNonPartitionedOperatorsState, - ChainedStateHandle chainedOperatorStateBackend, - ChainedStateHandle chainedOperatorStateStream, - KeyedStateHandle keyedStateHandleBackend, - KeyedStateHandle keyedStateHandleStream) { - - boolean hasAnyState = keyedStateHandleBackend != null - || keyedStateHandleStream != null - || !chainedOperatorStateBackend.isEmpty() - || !chainedOperatorStateStream.isEmpty() - || !chainedNonPartitionedOperatorsState.isEmpty(); - - // we signal a stateless task by reporting null, so that there are no attempts to assign empty state to - // stateless tasks on restore. This allows for simple job modifications that only concern stateless without - // the need to assign them uids to match their (always empty) states. - return hasAnyState ? new SubtaskState( - chainedNonPartitionedOperatorsState, - chainedOperatorStateBackend, - chainedOperatorStateStream, - keyedStateHandleBackend, - keyedStateHandleStream) - : null; - } - private void cleanup() throws Exception { if (asyncCheckpointState.compareAndSet(CheckpointingOperation.AsynCheckpointState.RUNNING, CheckpointingOperation.AsynCheckpointState.DISCARDED)) { LOG.debug("Cleanup AsyncCheckpointRunnable for checkpoint {} of {}.", checkpointMetaData.getCheckpointId(), owner.getName()); Exception exception = null; // clean up ongoing operator snapshot results and non partitioned state handles - for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) { + for (OperatorSnapshotResult operatorSnapshotResult : operatorSnapshotsInProgress.values()) { if (operatorSnapshotResult != null) { try { operatorSnapshotResult.cancel(); @@ -1031,7 +963,7 @@ private void cleanup() throws Exception { // discard non partitioned state handles try { - StateUtil.bestEffortDiscardAllStateObjects(nonPartitionedStateHandles); + StateUtil.bestEffortDiscardAllStateObjects(nonPartitionedStateHandles.values()); } catch (Exception discardException) { exception = ExceptionUtils.firstOrSuppressed(discardException, exception); } @@ -1069,8 +1001,8 @@ private static final class CheckpointingOperation { // ------------------------ - private final List nonPartitionedStates; - private final List snapshotInProgressList; + private final Map nonPartitionedStates; + private final Map operatorSnapshotsInProgress; public CheckpointingOperation( StreamTask owner, @@ -1083,8 +1015,8 @@ public CheckpointingOperation( this.checkpointOptions = Preconditions.checkNotNull(checkpointOptions); this.checkpointMetrics = Preconditions.checkNotNull(checkpointMetrics); this.allOperators = owner.operatorChain.getAllOperators(); - this.nonPartitionedStates = new ArrayList<>(allOperators.length); - this.snapshotInProgressList = new ArrayList<>(allOperators.length); + this.nonPartitionedStates = new HashMap<>(allOperators.length); + this.operatorSnapshotsInProgress = new HashMap<>(allOperators.length); } public void executeCheckpointing() throws Exception { @@ -1119,7 +1051,7 @@ public void executeCheckpointing() throws Exception { } finally { if (failed) { // Cleanup to release resources - for (OperatorSnapshotResult operatorSnapshotResult : snapshotInProgressList) { + for (OperatorSnapshotResult operatorSnapshotResult : operatorSnapshotsInProgress.values()) { if (null != operatorSnapshotResult) { try { operatorSnapshotResult.cancel(); @@ -1130,7 +1062,7 @@ public void executeCheckpointing() throws Exception { } // Cleanup non partitioned state handles - for (StreamStateHandle nonPartitionedState : nonPartitionedStates) { + for (StreamStateHandle nonPartitionedState : nonPartitionedStates.values()) { if (nonPartitionedState != null) { try { nonPartitionedState.discardState(); @@ -1156,21 +1088,19 @@ public void executeCheckpointing() throws Exception { private void checkpointStreamOperator(StreamOperator op) throws Exception { if (null != op) { // first call the legacy checkpoint code paths - nonPartitionedStates.add(op.snapshotLegacyOperatorState( - checkpointMetaData.getCheckpointId(), - checkpointMetaData.getTimestamp(), - checkpointOptions)); + StreamStateHandle legacyOperatorState = op.snapshotLegacyOperatorState( + checkpointMetaData.getCheckpointId(), + checkpointMetaData.getTimestamp(), + checkpointOptions); + + OperatorID operatorID = op.getOperatorID(); + nonPartitionedStates.put(operatorID, legacyOperatorState); OperatorSnapshotResult snapshotInProgress = op.snapshotState( checkpointMetaData.getCheckpointId(), checkpointMetaData.getTimestamp(), checkpointOptions); - - snapshotInProgressList.add(snapshotInProgress); - } else { - nonPartitionedStates.add(null); - OperatorSnapshotResult emptySnapshotInProgress = new OperatorSnapshotResult(); - snapshotInProgressList.add(emptySnapshotInProgress); + operatorSnapshotsInProgress.put(operatorID, snapshotInProgress); } } @@ -1179,7 +1109,7 @@ public void runAsyncCheckpointingAndAcknowledge() throws IOException { AsyncCheckpointRunnable asyncCheckpointRunnable = new AsyncCheckpointRunnable( owner, nonPartitionedStates, - snapshotInProgressList, + operatorSnapshotsInProgress, checkpointMetaData, checkpointMetrics, startAsyncPartNano); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java index e8b4c9e83c98e..ff5f589302f06 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.runtime.state.StreamStateHandle; @@ -84,7 +85,7 @@ public class AbstractUdfStreamOperatorLifecycleTest { "UDF::close"); private static final String ALL_METHODS_STREAM_OPERATOR = "[close[], dispose[], getChainingStrategy[], " + - "getMetricGroup[], initializeState[class org.apache.flink.streaming.runtime.tasks.OperatorStateHandles], " + + "getMetricGroup[], getOperatorID[], initializeState[class org.apache.flink.runtime.checkpoint.OperatorSubtaskState], " + "notifyOfCompletedCheckpoint[long], open[], setChainingStrategy[class " + "org.apache.flink.streaming.api.operators.ChainingStrategy], setKeyContextElement1[class " + "org.apache.flink.streaming.runtime.streamrecord.StreamRecord], " + @@ -132,6 +133,7 @@ public void testLifeCycleFull() throws Exception { MockSourceFunction srcFun = new MockSourceFunction(); cfg.setStreamOperator(new LifecycleTrackingStreamSource(srcFun, true)); + cfg.setOperatorID(new OperatorID()); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); Task task = StreamTaskTest.createTask(SourceStreamTask.class, cfg, taskManagerConfig); @@ -154,6 +156,7 @@ public void testLifeCycleCancel() throws Exception { StreamConfig cfg = new StreamConfig(new Configuration()); MockSourceFunction srcFun = new MockSourceFunction(); cfg.setStreamOperator(new LifecycleTrackingStreamSource(srcFun, false)); + cfg.setOperatorID(new OperatorID()); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); Task task = StreamTaskTest.createTask(SourceStreamTask.class, cfg, taskManagerConfig); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java index f9a1cd00ed091..1dd99fe040b36 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperatorTest.java @@ -29,15 +29,15 @@ 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.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.streaming.api.datastream.AsyncDataStream; @@ -500,7 +500,9 @@ public void testStateSnapshotAndRestore() throws Exception { AsyncDataStream.OutputMode.ORDERED); final StreamConfig streamConfig = testHarness.getStreamConfig(); + OperatorID operatorID = new OperatorID(42L, 4711L); streamConfig.setStreamOperator(operator); + streamConfig.setOperatorID(operatorID); final AcknowledgeStreamMockEnvironment env = new AcknowledgeStreamMockEnvironment( testHarness.jobConfig, @@ -540,7 +542,8 @@ public void testStateSnapshotAndRestore() throws Exception { // set the operator state from previous attempt into the restored one final OneInputStreamTask restoredTask = new OneInputStreamTask<>(); - restoredTask.setInitialState(new TaskStateHandles(env.getCheckpointStateHandles())); + TaskStateSnapshot subtaskStates = env.getCheckpointStateHandles(); + restoredTask.setInitialState(subtaskStates); final OneInputStreamTaskTestHarness restoredTaskHarness = new OneInputStreamTaskTestHarness<>(restoredTask, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO); @@ -553,6 +556,7 @@ public void testStateSnapshotAndRestore() throws Exception { AsyncDataStream.OutputMode.ORDERED); restoredTaskHarness.getStreamConfig().setStreamOperator(restoredOperator); + restoredTaskHarness.getStreamConfig().setOperatorID(operatorID); restoredTaskHarness.invoke(); restoredTaskHarness.waitForTaskRunning(); @@ -595,7 +599,7 @@ public void testStateSnapshotAndRestore() throws Exception { private static class AcknowledgeStreamMockEnvironment extends StreamMockEnvironment { private volatile long checkpointId; - private volatile SubtaskState checkpointStateHandles; + private volatile TaskStateSnapshot checkpointStateHandles; private final OneShotLatch checkpointLatch = new OneShotLatch(); @@ -614,7 +618,7 @@ public long getCheckpointId() { public void acknowledgeCheckpoint( long checkpointId, CheckpointMetrics checkpointMetrics, - SubtaskState checkpointStateHandles) { + TaskStateSnapshot checkpointStateHandles) { this.checkpointId = checkpointId; this.checkpointStateHandles = checkpointStateHandles; @@ -625,7 +629,7 @@ public OneShotLatch getCheckpointLatch() { return checkpointLatch; } - public SubtaskState getCheckpointStateHandles() { + public TaskStateSnapshot getCheckpointStateHandles() { return checkpointStateHandles; } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java index c2cf7f3f91ed5..491b23d17b057 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java @@ -23,6 +23,7 @@ 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.TaskStateSnapshot; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineOnCancellationBarrierException; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineSubsumedException; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -34,7 +35,6 @@ import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; -import org.apache.flink.runtime.state.TaskStateHandles; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; @@ -1484,7 +1484,7 @@ long getLastReportedBytesBufferedInAlignment() { } @Override - public void setInitialState(TaskStateHandles taskStateHandles) throws Exception { + public void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception { throw new UnsupportedOperationException("should never be called"); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java index 847db5cec006f..cde90104b91ac 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java @@ -22,13 +22,13 @@ 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.TaskStateSnapshot; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent; import org.apache.flink.runtime.jobgraph.tasks.StatefulTask; -import org.apache.flink.runtime.state.TaskStateHandles; import org.junit.Test; @@ -498,7 +498,7 @@ private CheckpointSequenceValidator(long... checkpointIDs) { } @Override - public void setInitialState(TaskStateHandles taskStateHandles) throws Exception { + public void setInitialState(TaskStateSnapshot taskStateHandles) throws Exception { throw new UnsupportedOperationException("should never be called"); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java index 6e3be0365fc33..65e59f8ac756c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamMap; import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; @@ -53,6 +54,7 @@ public void testOpenCloseAndTimestamps() throws Exception { StreamMap mapOperator = new StreamMap<>(new DummyMapFunction()); streamConfig.setStreamOperator(mapOperator); + streamConfig.setOperatorID(new OperatorID()); testHarness.invoke(); testHarness.waitForTaskRunning(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java index 675ffa3570ba7..d621b0bb12adb 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TestProcessingTimeServiceTest.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.operators; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.StreamMap; import org.apache.flink.streaming.runtime.tasks.AsyncExceptionHandler; @@ -53,6 +54,7 @@ public void testCustomTimeServiceProvider() throws Throwable { StreamMap mapOperator = new StreamMap<>(new StreamTaskTimerTest.DummyMapFunction()); streamConfig.setStreamOperator(mapOperator); + streamConfig.setOperatorID(new OperatorID()); testHarness.invoke(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java index 51328abbebc12..3b8178bb16fbf 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java @@ -45,6 +45,7 @@ import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; @@ -93,6 +94,7 @@ public void testBlockingNonInterruptibleCheckpoint() throws Exception { Configuration taskConfig = new Configuration(); StreamConfig cfg = new StreamConfig(taskConfig); cfg.setStreamOperator(new TestOperator()); + cfg.setOperatorID(new OperatorID()); cfg.setStateBackend(new LockingStreamStateBackend()); Task task = createTask(taskConfig); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index 25b504b916c16..82e4f3174fb42 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -26,6 +26,8 @@ import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; @@ -40,11 +42,11 @@ import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.DefaultOperatorStateBackend; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; @@ -55,7 +57,6 @@ import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerActions; @@ -135,18 +136,18 @@ private void testRestoreWithInterrupt(int mode) throws Exception { IN_RESTORE_LATCH.reset(); Configuration taskConfig = new Configuration(); - StreamConfig cfg = new StreamConfig(taskConfig); - cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + StreamConfig streamConfig = new StreamConfig(taskConfig); + streamConfig.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); switch (mode) { case OPERATOR_MANAGED: case OPERATOR_RAW: case KEYED_MANAGED: case KEYED_RAW: - cfg.setStateKeySerializer(IntSerializer.INSTANCE); - cfg.setStreamOperator(new StreamSource<>(new TestSource())); + streamConfig.setStateKeySerializer(IntSerializer.INSTANCE); + streamConfig.setStreamOperator(new StreamSource<>(new TestSource())); break; case LEGACY: - cfg.setStreamOperator(new StreamSource<>(new TestSourceLegacy())); + streamConfig.setStreamOperator(new StreamSource<>(new TestSourceLegacy())); break; default: throw new IllegalArgumentException(); @@ -154,7 +155,7 @@ private void testRestoreWithInterrupt(int mode) throws Exception { StreamStateHandle lockingHandle = new InterruptLockingStateHandle(); - Task task = createTask(taskConfig, lockingHandle, mode); + Task task = createTask(streamConfig, taskConfig, lockingHandle, mode); // start the task and wait until it is in "restore" task.startTaskThread(); @@ -178,19 +179,20 @@ private void testRestoreWithInterrupt(int mode) throws Exception { // ------------------------------------------------------------------------ private static Task createTask( - Configuration taskConfig, - StreamStateHandle state, - int mode) throws IOException { + StreamConfig streamConfig, + Configuration taskConfig, + StreamStateHandle state, + int mode) throws IOException { NetworkEnvironment networkEnvironment = mock(NetworkEnvironment.class); when(networkEnvironment.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class))) .thenReturn(mock(TaskKvStateRegistry.class)); - ChainedStateHandle operatorState = null; - List keyedStateFromBackend = Collections.emptyList(); - List keyedStateFromStream = Collections.emptyList(); - List> operatorStateBackend = Collections.emptyList(); - List> operatorStateStream = Collections.emptyList(); + StreamStateHandle operatorState = null; + Collection keyedStateFromBackend = Collections.emptyList(); + Collection keyedStateFromStream = Collections.emptyList(); + Collection operatorStateBackend = Collections.emptyList(); + Collection operatorStateStream = Collections.emptyList(); Map operatorStateMetadata = new HashMap<>(1); OperatorStateHandle.StateMetaInfo metaInfo = @@ -207,10 +209,10 @@ private static Task createTask( switch (mode) { case OPERATOR_MANAGED: - operatorStateBackend = Collections.singletonList(operatorStateHandles); + operatorStateBackend = operatorStateHandles; break; case OPERATOR_RAW: - operatorStateStream = Collections.singletonList(operatorStateHandles); + operatorStateStream = operatorStateHandles; break; case KEYED_MANAGED: keyedStateFromBackend = keyedStateHandles; @@ -219,29 +221,35 @@ private static Task createTask( keyedStateFromStream = keyedStateHandles; break; case LEGACY: - operatorState = new ChainedStateHandle<>(Collections.singletonList(state)); + operatorState = state; break; default: throw new IllegalArgumentException(); } - TaskStateHandles taskStateHandles = new TaskStateHandles( + OperatorSubtaskState operatorSubtaskState = new OperatorSubtaskState( operatorState, operatorStateBackend, operatorStateStream, keyedStateFromBackend, keyedStateFromStream); + JobVertexID jobVertexID = new JobVertexID(); + OperatorID operatorID = OperatorID.fromJobVertexID(jobVertexID); + streamConfig.setOperatorID(operatorID); + + TaskStateSnapshot stateSnapshot = new TaskStateSnapshot(); + stateSnapshot.putSubtaskStateByOperatorID(operatorID, operatorSubtaskState); JobInformation jobInformation = new JobInformation( new JobID(), "test job name", new SerializedValue<>(new ExecutionConfig()), - new Configuration(), + taskConfig, Collections.emptyList(), Collections.emptyList()); TaskInformation taskInformation = new TaskInformation( - new JobVertexID(), + jobVertexID, "test task name", 1, 1, @@ -258,7 +266,7 @@ private static Task createTask( Collections.emptyList(), Collections.emptyList(), 0, - taskStateHandles, + stateSnapshot, mock(MemoryManager.class), mock(IOManager.class), networkEnvironment, @@ -273,7 +281,6 @@ private static Task createTask( mock(ResultPartitionConsumableNotifier.class), mock(PartitionProducerStateChecker.class), mock(Executor.class)); - } // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index f7987a122589f..3190620111b48 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -34,13 +34,13 @@ 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.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.streaming.api.collector.selector.OutputSelector; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.graph.StreamEdge; @@ -109,6 +109,7 @@ public void testOpenCloseAndTimestamps() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); StreamMap mapOperator = new StreamMap(new TestOpenCloseMapFunction()); streamConfig.setStreamOperator(mapOperator); + streamConfig.setOperatorID(new OperatorID()); long initialTime = 0L; ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); @@ -151,6 +152,7 @@ public void testWatermarkAndStreamStatusForwarding() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); StreamMap mapOperator = new StreamMap(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); + streamConfig.setOperatorID(new OperatorID()); ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; @@ -261,15 +263,21 @@ public void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { // ------------------ setup the chain ------------------ TriggerableFailOnWatermarkTestOperator headOperator = new TriggerableFailOnWatermarkTestOperator(); + OperatorID headOperatorId = new OperatorID(); + StreamConfig headOperatorConfig = testHarness.getStreamConfig(); WatermarkGeneratingTestOperator watermarkOperator = new WatermarkGeneratingTestOperator(); + OperatorID watermarkOperatorId = new OperatorID(); + StreamConfig watermarkOperatorConfig = new StreamConfig(new Configuration()); TriggerableFailOnWatermarkTestOperator tailOperator = new TriggerableFailOnWatermarkTestOperator(); + OperatorID tailOperatorId = new OperatorID(); StreamConfig tailOperatorConfig = new StreamConfig(new Configuration()); headOperatorConfig.setStreamOperator(headOperator); + headOperatorConfig.setOperatorID(headOperatorId); headOperatorConfig.setChainStart(); headOperatorConfig.setChainIndex(0); headOperatorConfig.setChainedOutputs(Collections.singletonList(new StreamEdge( @@ -282,6 +290,7 @@ public void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { ))); watermarkOperatorConfig.setStreamOperator(watermarkOperator); + watermarkOperatorConfig.setOperatorID(watermarkOperatorId); watermarkOperatorConfig.setTypeSerializerIn1(StringSerializer.INSTANCE); watermarkOperatorConfig.setChainIndex(1); watermarkOperatorConfig.setChainedOutputs(Collections.singletonList(new StreamEdge( @@ -303,6 +312,7 @@ public void testWatermarksNotForwardedWithinChainWhenIdle() throws Exception { null)); tailOperatorConfig.setStreamOperator(tailOperator); + tailOperatorConfig.setOperatorID(tailOperatorId); tailOperatorConfig.setTypeSerializerIn1(StringSerializer.INSTANCE); tailOperatorConfig.setBufferTimeout(0); tailOperatorConfig.setChainIndex(2); @@ -412,6 +422,7 @@ public void testCheckpointBarriers() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); StreamMap mapOperator = new StreamMap(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); + streamConfig.setOperatorID(new OperatorID()); ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; @@ -471,6 +482,7 @@ public void testOvertakingCheckpointBarriers() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); StreamMap mapOperator = new StreamMap(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); + streamConfig.setOperatorID(new OperatorID()); ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; @@ -580,15 +592,20 @@ public void testSnapshottingAndRestoring() throws Exception { testHarness.waitForTaskCompletion(deadline.timeLeft().toMillis()); final OneInputStreamTask restoredTask = new OneInputStreamTask(); - restoredTask.setInitialState(new TaskStateHandles(env.getCheckpointStateHandles())); - final OneInputStreamTaskTestHarness restoredTaskHarness = new OneInputStreamTaskTestHarness(restoredTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); + final OneInputStreamTaskTestHarness restoredTaskHarness = + new OneInputStreamTaskTestHarness(restoredTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO); restoredTaskHarness.configureForKeyedStream(keySelector, BasicTypeInfo.STRING_TYPE_INFO); StreamConfig restoredTaskStreamConfig = restoredTaskHarness.getStreamConfig(); configureChainedTestingStreamOperator(restoredTaskStreamConfig, numberChainedTasks, seed, recoveryTimestamp); + TaskStateSnapshot stateHandles = env.getCheckpointStateHandles(); + Assert.assertEquals(numberChainedTasks, stateHandles.getSubtaskStateMappings().size()); + + restoredTask.setInitialState(stateHandles); + TestingStreamOperator.numberRestoreCalls = 0; restoredTaskHarness.invoke(); @@ -601,6 +618,7 @@ public void testSnapshottingAndRestoring() throws Exception { TestingStreamOperator.numberRestoreCalls = 0; } + //============================================================================================== // Utility functions and classes //============================================================================================== @@ -618,6 +636,7 @@ private void configureChainedTestingStreamOperator( TestingStreamOperator previousOperator = new TestingStreamOperator<>(random.nextLong(), recoveryTimestamp); streamConfig.setStreamOperator(previousOperator); + streamConfig.setOperatorID(new OperatorID(0L, 0L)); // create the chain of operators Map chainedTaskConfigs = new HashMap<>(numberChainedTasks - 1); @@ -627,6 +646,7 @@ private void configureChainedTestingStreamOperator( TestingStreamOperator chainedOperator = new TestingStreamOperator<>(random.nextLong(), recoveryTimestamp); StreamConfig chainedConfig = new StreamConfig(new Configuration()); chainedConfig.setStreamOperator(chainedOperator); + chainedConfig.setOperatorID(new OperatorID(0L, chainedIndex)); chainedTaskConfigs.put(chainedIndex, chainedConfig); StreamEdge outputEdge = new StreamEdge( @@ -673,7 +693,7 @@ public IN getKey(IN value) throws Exception { private static class AcknowledgeStreamMockEnvironment extends StreamMockEnvironment { private volatile long checkpointId; - private volatile SubtaskState checkpointStateHandles; + private volatile TaskStateSnapshot checkpointStateHandles; private final OneShotLatch checkpointLatch = new OneShotLatch(); @@ -692,7 +712,7 @@ public long getCheckpointId() { public void acknowledgeCheckpoint( long checkpointId, CheckpointMetrics checkpointMetrics, - SubtaskState checkpointStateHandles) { + TaskStateSnapshot checkpointStateHandles) { this.checkpointId = checkpointId; this.checkpointStateHandles = checkpointStateHandles; @@ -703,7 +723,7 @@ public OneShotLatch getCheckpointLatch() { return checkpointLatch; } - public SubtaskState getCheckpointStateHandles() { + public TaskStateSnapshot getCheckpointStateHandles() { return checkpointStateHandles; } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceExternalCheckpointTriggerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceExternalCheckpointTriggerTest.java index 47a53500be822..b3b0a9f414e35 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceExternalCheckpointTriggerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceExternalCheckpointTriggerTest.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.MasterTriggerRestoreHook; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.checkpoint.ExternallyInducedSource; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -64,6 +65,7 @@ public void testCheckpointsTriggeredBySource() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); StreamSource sourceOperator = new StreamSource<>(source); streamConfig.setStreamOperator(sourceOperator); + streamConfig.setOperatorID(new OperatorID()); // this starts the source thread testHarness.invoke(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index 27818bcafa36d..8867632a5c3ff 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -63,6 +64,7 @@ public void testOpenClose() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); StreamSource sourceOperator = new StreamSource<>(new OpenCloseTestSource()); streamConfig.setStreamOperator(sourceOperator); + streamConfig.setOperatorID(new OperatorID()); testHarness.invoke(); testHarness.waitForTaskCompletion(); @@ -106,6 +108,7 @@ public void testCheckpointing() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); StreamSource, ?> sourceOperator = new StreamSource<>(new MockSource(numElements, sourceCheckpointDelay, sourceReadDelay)); streamConfig.setStreamOperator(sourceOperator); + streamConfig.setOperatorID(new OperatorID()); // prepare the diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java index 5b995c67b8e94..231f59e97fb2a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; -import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -333,7 +333,7 @@ public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpoin } @Override - public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, SubtaskState subtaskState) { + public void acknowledgeCheckpoint(long checkpointId, CheckpointMetrics checkpointMetrics, TaskStateSnapshot subtaskState) { } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java index 6e3c299f9ed9d..36bdc054b9340 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationBarrierTest.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.decline.CheckpointDeclineOnCancellationBarrierException; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; @@ -91,6 +92,7 @@ public void testDeclineCallOnCancelBarrierOneInput() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); StreamMap mapOperator = new StreamMap<>(new IdentityMap()); streamConfig.setStreamOperator(mapOperator); + streamConfig.setOperatorID(new OperatorID()); StreamMockEnvironment environment = spy(testHarness.createEnvironment()); @@ -135,6 +137,7 @@ public void testDeclineCallOnCancelBarrierTwoInputs() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); CoStreamMap op = new CoStreamMap<>(new UnionCoMap()); streamConfig.setStreamOperator(op); + streamConfig.setOperatorID(new OperatorID()); StreamMockEnvironment environment = spy(testHarness.createEnvironment()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 4f2135d96bc0f..702d833ad19ae 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -42,6 +42,7 @@ import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; @@ -107,6 +108,7 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E final AbstractStateBackend blockingStateBackend = new BlockingStateBackend(); streamConfig.setStreamOperator(noOpStreamOperator); + streamConfig.setOperatorID(new OperatorID()); streamConfig.setStateBackend(blockingStateBackend); final long checkpointId = 0L; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 923b912437a2c..09e9a1b26144c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -32,7 +32,9 @@ 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.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.SubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; @@ -49,6 +51,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; @@ -56,7 +59,6 @@ import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.AbstractStateBackend; -import org.apache.flink.runtime.state.ChainedStateHandle; import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.DoneFuture; import org.apache.flink.runtime.state.KeyGroupRange; @@ -65,7 +67,6 @@ import org.apache.flink.runtime.state.OperatorStateHandle; import org.apache.flink.runtime.state.StateBackendFactory; import org.apache.flink.runtime.state.StreamStateHandle; -import org.apache.flink.runtime.state.TaskStateHandles; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskExecutionState; @@ -128,6 +129,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyCollectionOf; import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; @@ -158,6 +160,7 @@ public class StreamTaskTest extends TestLogger { public void testEarlyCanceling() throws Exception { Deadline deadline = new FiniteDuration(2, TimeUnit.MINUTES).fromNow(); StreamConfig cfg = new StreamConfig(new Configuration()); + cfg.setOperatorID(new OperatorID(4711L, 42L)); cfg.setStreamOperator(new SlowlyDeserializingOperator()); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); @@ -203,6 +206,7 @@ public void testStateBackendLoadingAndClosing() throws Exception { taskManagerConfig.setString(CoreOptions.STATE_BACKEND, MockStateBackend.class.getName()); StreamConfig cfg = new StreamConfig(new Configuration()); + cfg.setOperatorID(new OperatorID(4711L, 42L)); cfg.setStreamOperator(new StreamSource<>(new MockSourceFunction())); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); @@ -227,6 +231,7 @@ public void testStateBackendClosingOnFailure() throws Exception { taskManagerConfig.setString(CoreOptions.STATE_BACKEND, MockStateBackend.class.getName()); StreamConfig cfg = new StreamConfig(new Configuration()); + cfg.setOperatorID(new OperatorID(4711L, 42L)); cfg.setStreamOperator(new StreamSource<>(new MockSourceFunction())); cfg.setTimeCharacteristic(TimeCharacteristic.ProcessingTime); @@ -324,6 +329,13 @@ public void testFailingCheckpointStreamOperator() throws Exception { when(streamOperator2.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle2); when(streamOperator3.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle3); + OperatorID operatorID1 = new OperatorID(); + OperatorID operatorID2 = new OperatorID(); + OperatorID operatorID3 = new OperatorID(); + when(streamOperator1.getOperatorID()).thenReturn(operatorID1); + when(streamOperator2.getOperatorID()).thenReturn(operatorID2); + when(streamOperator3.getOperatorID()).thenReturn(operatorID3); + // set up the task StreamOperator[] streamOperators = {streamOperator1, streamOperator2, streamOperator3}; @@ -399,6 +411,13 @@ public void testFailingAsyncCheckpointRunnable() throws Exception { when(streamOperator2.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle2); when(streamOperator3.snapshotLegacyOperatorState(anyLong(), anyLong(), any(CheckpointOptions.class))).thenReturn(streamStateHandle3); + OperatorID operatorID1 = new OperatorID(); + OperatorID operatorID2 = new OperatorID(); + OperatorID operatorID3 = new OperatorID(); + when(streamOperator1.getOperatorID()).thenReturn(operatorID1); + when(streamOperator2.getOperatorID()).thenReturn(operatorID2); + when(streamOperator3.getOperatorID()).thenReturn(operatorID3); + StreamOperator[] streamOperators = {streamOperator1, streamOperator2, streamOperator3}; OperatorChain> operatorChain = mock(OperatorChain.class); @@ -455,7 +474,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { return null; } - }).when(mockEnvironment).acknowledgeCheckpoint(anyLong(), any(CheckpointMetrics.class), any(SubtaskState.class)); + }).when(mockEnvironment).acknowledgeCheckpoint(anyLong(), any(CheckpointMetrics.class), any(TaskStateSnapshot.class)); StreamTask> streamTask = mock(StreamTask.class, Mockito.CALLS_REAL_METHODS); CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); @@ -505,18 +524,19 @@ public Object answer(InvocationOnMock invocation) throws Throwable { acknowledgeCheckpointLatch.await(); - ArgumentCaptor subtaskStateCaptor = ArgumentCaptor.forClass(SubtaskState.class); + ArgumentCaptor subtaskStateCaptor = ArgumentCaptor.forClass(TaskStateSnapshot.class); // check that the checkpoint has been completed verify(mockEnvironment).acknowledgeCheckpoint(eq(checkpointId), any(CheckpointMetrics.class), subtaskStateCaptor.capture()); - SubtaskState subtaskState = subtaskStateCaptor.getValue(); + TaskStateSnapshot subtaskStates = subtaskStateCaptor.getValue(); + OperatorSubtaskState subtaskState = subtaskStates.getSubtaskStateMappings().iterator().next().getValue(); // check that the subtask state contains the expected state handles - assertEquals(managedKeyedStateHandle, subtaskState.getManagedKeyedState()); - assertEquals(rawKeyedStateHandle, subtaskState.getRawKeyedState()); - assertEquals(new ChainedStateHandle<>(Collections.singletonList(managedOperatorStateHandle)), subtaskState.getManagedOperatorState()); - assertEquals(new ChainedStateHandle<>(Collections.singletonList(rawOperatorStateHandle)), subtaskState.getRawOperatorState()); + assertEquals(Collections.singletonList(managedKeyedStateHandle), subtaskState.getManagedKeyedState()); + assertEquals(Collections.singletonList(rawKeyedStateHandle), subtaskState.getRawKeyedState()); + assertEquals(Collections.singletonList(managedOperatorStateHandle), subtaskState.getManagedOperatorState()); + assertEquals(Collections.singletonList(rawOperatorStateHandle), subtaskState.getRawOperatorState()); // check that the state handles have not been discarded verify(managedKeyedStateHandle, never()).discardState(); @@ -558,18 +578,26 @@ public void testAsyncCheckpointingConcurrentCloseBeforeAcknowledge() throws Exce Environment mockEnvironment = mock(Environment.class); when(mockEnvironment.getTaskInfo()).thenReturn(mockTaskInfo); - whenNew(SubtaskState.class).withAnyArguments().thenAnswer(new Answer() { - @Override - public SubtaskState answer(InvocationOnMock invocation) throws Throwable { + whenNew(OperatorSubtaskState.class). + withArguments( + any(StreamStateHandle.class), + anyCollectionOf(OperatorStateHandle.class), + anyCollectionOf(OperatorStateHandle.class), + anyCollectionOf(KeyedStateHandle.class), + anyCollectionOf(KeyedStateHandle.class)). + thenAnswer(new Answer() { + @Override + public OperatorSubtaskState answer(InvocationOnMock invocation) throws Throwable { createSubtask.trigger(); completeSubtask.await(); - - return new SubtaskState( - (ChainedStateHandle) invocation.getArguments()[0], - (ChainedStateHandle) invocation.getArguments()[1], - (ChainedStateHandle) invocation.getArguments()[2], - (KeyedStateHandle) invocation.getArguments()[3], - (KeyedStateHandle) invocation.getArguments()[4]); + Object[] arguments = invocation.getArguments(); + return new OperatorSubtaskState( + (StreamStateHandle) arguments[0], + (OperatorStateHandle) arguments[1], + (OperatorStateHandle) arguments[2], + (KeyedStateHandle) arguments[3], + (KeyedStateHandle) arguments[4] + ); } }); @@ -577,7 +605,9 @@ public SubtaskState answer(InvocationOnMock invocation) throws Throwable { CheckpointMetaData checkpointMetaData = new CheckpointMetaData(checkpointId, timestamp); streamTask.setEnvironment(mockEnvironment); - StreamOperator streamOperator = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class)); + final StreamOperator streamOperator = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class)); + final OperatorID operatorID = new OperatorID(); + when(streamOperator.getOperatorID()).thenReturn(operatorID); KeyedStateHandle managedKeyedStateHandle = mock(KeyedStateHandle.class); KeyedStateHandle rawKeyedStateHandle = mock(KeyedStateHandle.class); @@ -636,7 +666,7 @@ public SubtaskState answer(InvocationOnMock invocation) throws Throwable { } // check that the checkpoint has not been acknowledged - verify(mockEnvironment, never()).acknowledgeCheckpoint(eq(checkpointId), any(CheckpointMetrics.class), any(SubtaskState.class)); + verify(mockEnvironment, never()).acknowledgeCheckpoint(eq(checkpointId), any(CheckpointMetrics.class), any(TaskStateSnapshot.class)); // check that the state handles have been discarded verify(managedKeyedStateHandle).discardState(); @@ -676,7 +706,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { checkpointCompletedLatch.trigger(); return null; } - }).when(mockEnvironment).acknowledgeCheckpoint(anyLong(), any(CheckpointMetrics.class), any(SubtaskState.class)); + }).when(mockEnvironment).acknowledgeCheckpoint(anyLong(), any(CheckpointMetrics.class), any(TaskStateSnapshot.class)); when(mockEnvironment.getTaskInfo()).thenReturn(mockTaskInfo); @@ -688,6 +718,9 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { StreamOperator statelessOperator = mock(StreamOperator.class, withSettings().extraInterfaces(StreamCheckpointedOperator.class)); + final OperatorID operatorID = new OperatorID(); + when(statelessOperator.getOperatorID()).thenReturn(operatorID); + // mock the returned empty snapshot result (all state handles are null) OperatorSnapshotResult statelessOperatorSnapshotResult = new OperatorSnapshotResult(); when(statelessOperator.snapshotState(anyLong(), anyLong(), any(CheckpointOptions.class))) @@ -803,7 +836,7 @@ public static Task createTask( Collections.emptyList(), Collections.emptyList(), 0, - new TaskStateHandles(), + new TaskStateSnapshot(), mock(MemoryManager.class), mock(IOManager.class), network, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java index a02fe4e7d9a56..19d48e195f2ef 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; @@ -142,6 +143,7 @@ public void setupOutputForSingletonOperatorChain() { streamConfig.setNumberOfOutputs(1); streamConfig.setTypeSerializerOut(outputSerializer); streamConfig.setVertexID(0); + streamConfig.setOperatorID(new OperatorID(4711L, 123L)); StreamOperator dummyOperator = new AbstractStreamOperator() { private static final long serialVersionUID = 1L; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java index 66531ac51ce28..d785c0d7517f5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.io.network.api.CancelCheckpointMarker; import org.apache.flink.runtime.io.network.api.CheckpointBarrier; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.functions.co.RichCoMapFunction; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -64,6 +65,7 @@ public void testOpenCloseAndTimestamps() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); CoStreamMap coMapOperator = new CoStreamMap(new TestOpenCloseMapFunction()); streamConfig.setStreamOperator(coMapOperator); + streamConfig.setOperatorID(new OperatorID()); long initialTime = 0L; ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); @@ -110,6 +112,7 @@ public void testWatermarkAndStreamStatusForwarding() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); streamConfig.setStreamOperator(coMapOperator); + streamConfig.setOperatorID(new OperatorID()); ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; @@ -216,6 +219,7 @@ public void testCheckpointBarriers() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); streamConfig.setStreamOperator(coMapOperator); + streamConfig.setOperatorID(new OperatorID()); ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; @@ -296,6 +300,7 @@ public void testOvertakingCheckpointBarriers() throws Exception { StreamConfig streamConfig = testHarness.getStreamConfig(); CoStreamMap coMapOperator = new CoStreamMap(new IdentityMap()); streamConfig.setStreamOperator(coMapOperator); + streamConfig.setOperatorID(new OperatorID()); ConcurrentLinkedQueue expectedOutput = new ConcurrentLinkedQueue(); long initialTime = 0L; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java index 47e8726874d33..15802353ab5c5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/AbstractStreamOperatorTestHarness.java @@ -32,9 +32,11 @@ import org.apache.flink.migration.util.MigrationInstantiationUtil; import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.checkpoint.OperatorStateRepartitioner; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.RoundRobinOperatorStateRepartitioner; import org.apache.flink.runtime.checkpoint.StateAssignmentOperation; import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.CheckpointStreamFactory; @@ -154,6 +156,7 @@ public AbstractStreamOperatorTestHarness( Configuration underlyingConfig = environment.getTaskConfiguration(); this.config = new StreamConfig(underlyingConfig); this.config.setCheckpointingEnabled(true); + this.config.setOperatorID(new OperatorID()); this.executionConfig = environment.getExecutionConfig(); this.closableRegistry = new CloseableRegistry(); this.checkpointLock = new Object(); @@ -336,7 +339,7 @@ public void initializeStateFromLegacyCheckpoint(String checkpointFilename) throw } /** - * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#initializeState(OperatorStateHandles)}. + * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#initializeState(OperatorSubtaskState)}. * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#setup(StreamTask, StreamConfig, Output)} * if it was not called before. * @@ -393,13 +396,12 @@ public void initializeState(OperatorStateHandles operatorStateHandles) throws Ex rawOperatorState, numSubtasks).get(subtaskIndex); - OperatorStateHandles massagedOperatorStateHandles = new OperatorStateHandles( - 0, - operatorStateHandles.getLegacyOperatorState(), - localManagedKeyGroupState, - localRawKeyGroupState, - localManagedOperatorState, - localRawOperatorState); + OperatorSubtaskState massagedOperatorStateHandles = new OperatorSubtaskState( + operatorStateHandles.getLegacyOperatorState(), + nullToEmptyCollection(localManagedOperatorState), + nullToEmptyCollection(localRawOperatorState), + nullToEmptyCollection(localManagedKeyGroupState), + nullToEmptyCollection(localRawKeyGroupState)); operator.initializeState(massagedOperatorStateHandles); } else { @@ -408,6 +410,10 @@ public void initializeState(OperatorStateHandles operatorStateHandles) throws Ex initializeCalled = true; } + private static Collection nullToEmptyCollection(Collection collection) { + return collection != null ? collection : Collections.emptyList(); + } + /** * Takes the different {@link OperatorStateHandles} created by calling {@link #snapshot(long, long)} * on different instances of {@link AbstractStreamOperatorTestHarness} (each one representing one subtask) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index bf1bb1bb987a9..cc235458c5e87 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -354,7 +354,7 @@ protected void run() { errMsg = "Initial operator state mismatch."; assertEquals(errMsg, subtaskState.getLegacyOperatorState(), - tdd.getTaskStateHandles().getLegacyOperatorState().get(chainIndexAndJobVertex.f0)); + tdd.getTaskStateHandles().getSubtaskStateByOperatorID(operatorState.getOperatorID()).getLegacyOperatorState()); } } From 91a4b276171afb760bfff9ccf30593e648e91dfb Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Tue, 25 Jul 2017 12:04:16 +0200 Subject: [PATCH 021/129] [FLINK-7268] [checkpoints] Scope SharedStateRegistry objects per (re)start --- .../state/RocksDBKeyedStateBackend.java | 13 +- .../checkpoint/CheckpointCoordinator.java | 32 +- .../checkpoint/CompletedCheckpoint.java | 2 + .../checkpoint/CompletedCheckpointStore.java | 5 +- .../StandaloneCompletedCheckpointStore.java | 4 +- .../ZooKeeperCompletedCheckpointStore.java | 12 +- .../executiongraph/ExecutionGraph.java | 6 +- .../executiongraph/ExecutionJobVertex.java | 2 +- .../flink/runtime/jobmaster/JobMaster.java | 2 +- .../state/IncrementalKeyedStateHandle.java | 68 +++- .../runtime/state/KeyGroupsStateHandle.java | 2 +- .../runtime/state/MultiStreamStateHandle.java | 10 +- .../runtime/state/SharedStateRegistry.java | 52 ++- .../state/SharedStateRegistryFactory.java | 35 ++ .../state/memory/ByteStreamStateHandle.java | 1 + ...oordinatorExternalizedCheckpointsTest.java | 22 +- .../CheckpointCoordinatorFailureTest.java | 7 +- .../CheckpointCoordinatorMasterHooksTest.java | 7 +- .../checkpoint/CheckpointCoordinatorTest.java | 341 ++++++++++++++++-- .../CheckpointStateRestoreTest.java | 10 +- ...oKeeperCompletedCheckpointStoreITCase.java | 25 +- ...ZooKeeperCompletedCheckpointStoreTest.java | 7 +- .../IncrementalKeyedStateHandleTest.java | 75 +++- .../RecoverableCompletedCheckpointStore.java | 33 +- .../streaming/runtime/tasks/StreamTask.java | 21 +- ...actEventTimeWindowCheckpointingITCase.java | 76 ++-- ...endEventTimeWindowCheckpointingITCase.java | 49 +++ 27 files changed, 743 insertions(+), 176 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/checkpointing/HAIncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.java diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index bba5b551527d5..756cfddf1d630 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -253,7 +253,10 @@ public RocksDBKeyedStateBackend( this.restoredKvStateMetaInfos = new HashMap<>(); this.materializedSstFiles = new TreeMap<>(); this.backendUID = UUID.randomUUID(); - LOG.debug("Setting initial keyed backend uid for operator {} to {}.", this.operatorIdentifier, this.backendUID); + + LOG.debug("Setting initial backend ID in RocksDBKeyedStateBackend for operator {} to {}.", + this.operatorIdentifier, + this.backendUID); } /** @@ -883,11 +886,17 @@ private StreamStateHandle materializeMetaData() throws Exception { void takeSnapshot() throws Exception { assert (Thread.holdsLock(stateBackend.asyncSnapshotLock)); + final long lastCompletedCheckpoint; + // use the last completed checkpoint as the comparison base. synchronized (stateBackend.materializedSstFiles) { - baseSstFiles = stateBackend.materializedSstFiles.get(stateBackend.lastCompletedCheckpointId); + lastCompletedCheckpoint = stateBackend.lastCompletedCheckpointId; + baseSstFiles = stateBackend.materializedSstFiles.get(lastCompletedCheckpoint); } + LOG.trace("Taking incremental snapshot for checkpoint {}. Snapshot is based on last completed checkpoint {} " + + "assuming the following (shared) files as base: {}.", checkpointId, lastCompletedCheckpoint, baseSstFiles); + // save meta data for (Map.Entry>> stateMetaInfoEntry : stateBackend.kvStateInformation.entrySet()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 0b64a737fef7b..c98d3aa1083c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -40,6 +40,7 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.SharedStateRegistryFactory; import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; @@ -174,8 +175,11 @@ public class CheckpointCoordinator { @Nullable private CheckpointStatsTracker statsTracker; + /** A factory for SharedStateRegistry objects */ + private final SharedStateRegistryFactory sharedStateRegistryFactory; + /** Registry that tracks state which is shared across (incremental) checkpoints */ - private final SharedStateRegistry sharedStateRegistry; + private SharedStateRegistry sharedStateRegistry; // -------------------------------------------------------------------------------------------- @@ -192,7 +196,8 @@ public CheckpointCoordinator( CheckpointIDCounter checkpointIDCounter, CompletedCheckpointStore completedCheckpointStore, @Nullable String checkpointDirectory, - Executor executor) { + Executor executor, + SharedStateRegistryFactory sharedStateRegistryFactory) { // sanity checks checkArgument(baseInterval > 0, "Checkpoint timeout must be larger than zero"); @@ -230,7 +235,8 @@ public CheckpointCoordinator( this.completedCheckpointStore = checkNotNull(completedCheckpointStore); this.checkpointDirectory = checkpointDirectory; this.executor = checkNotNull(executor); - this.sharedStateRegistry = new SharedStateRegistry(executor); + this.sharedStateRegistryFactory = checkNotNull(sharedStateRegistryFactory); + this.sharedStateRegistry = sharedStateRegistryFactory.create(executor); this.recentPendingCheckpoints = new ArrayDeque<>(NUM_GHOST_CHECKPOINT_IDS); this.masterHooks = new HashMap<>(); @@ -1043,10 +1049,23 @@ public boolean restoreLatestCheckpointedState( throw new IllegalStateException("CheckpointCoordinator is shut down"); } - // Recover the checkpoints - completedCheckpointStore.recover(sharedStateRegistry); + // We create a new shared state registry object, so that all pending async disposal requests from previous + // runs will go against the old object (were they can do no harm). + // This must happen under the checkpoint lock. + sharedStateRegistry.close(); + sharedStateRegistry = sharedStateRegistryFactory.create(executor); + + // Recover the checkpoints, TODO this could be done only when there is a new leader, not on each recovery + completedCheckpointStore.recover(); + + // Now, we re-register all (shared) states from the checkpoint store with the new registry + for (CompletedCheckpoint completedCheckpoint : completedCheckpointStore.getAllCheckpoints()) { + completedCheckpoint.registerSharedStatesAfterRestored(sharedStateRegistry); + } + + LOG.debug("Status of the shared state registry after restore: {}.", sharedStateRegistry); - // restore from the latest checkpoint + // Restore from the latest checkpoint CompletedCheckpoint latest = completedCheckpointStore.getLatestCheckpoint(); if (latest == null) { @@ -1120,7 +1139,6 @@ public boolean restoreSavepoint( CompletedCheckpoint savepoint = SavepointLoader.loadAndValidateSavepoint( job, tasks, savepointPath, userClassLoader, allowNonRestored); - savepoint.registerSharedStatesAfterRestored(sharedStateRegistry); completedCheckpointStore.addCheckpoint(savepoint); // Reset the checkpoint ID counter 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 7c3edee081669..d3f61e448acc9 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 @@ -209,6 +209,8 @@ public boolean discardOnShutdown(JobStatus jobStatus) throws Exception { private void doDiscard() throws Exception { + LOG.trace("Executing discard procedure for {}.", this); + try { // collect exceptions and continue cleanup Exception exception = null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java index 45d407e91a045..82193b5f08da9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStore.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.state.SharedStateRegistry; import java.util.List; @@ -33,10 +32,8 @@ public interface CompletedCheckpointStore { * *

After a call to this method, {@link #getLatestCheckpoint()} returns the latest * available checkpoint. - * - * @param sharedStateRegistry the shared state registry to register recovered states. */ - void recover(SharedStateRegistry sharedStateRegistry) throws Exception; + void recover() throws Exception; /** * Adds a {@link CompletedCheckpoint} instance to the list of completed checkpoints. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java index fbb0198a7f17c..63e7468ebca74 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; -import org.apache.flink.runtime.state.SharedStateRegistry; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,7 +57,7 @@ public StandaloneCompletedCheckpointStore(int maxNumberOfCheckpointsToRetain) { } @Override - public void recover(SharedStateRegistry sharedStateRegistry) throws Exception { + public void recover() throws Exception { // Nothing to do } 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 c4cb6bca3a58d..88dd0d4a14f5e 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 @@ -18,20 +18,21 @@ package org.apache.flink.runtime.checkpoint; -import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.utils.ZKPaths; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.state.RetrievableStateHandle; -import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper; import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore; import org.apache.flink.util.FlinkException; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.utils.ZKPaths; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; + import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; @@ -138,14 +139,13 @@ public boolean requiresExternalizedCheckpoints() { * that the history of checkpoints is consistent. */ @Override - public void recover(SharedStateRegistry sharedStateRegistry) throws Exception { + public void recover() throws Exception { LOG.info("Recovering checkpoints from ZooKeeper."); // Clear local handles in order to prevent duplicates on // recovery. The local handles should reflect the state // of ZooKeeper. completedCheckpoints.clear(); - sharedStateRegistry.clear(); // Get all there is first List, String>> initialCheckpoints; @@ -170,8 +170,6 @@ public void recover(SharedStateRegistry sharedStateRegistry) throws Exception { try { completedCheckpoint = retrieveCompletedCheckpoint(checkpointStateHandle); if (completedCheckpoint != null) { - // Re-register all shared states in the checkpoint. - completedCheckpoint.registerSharedStatesAfterRestored(sharedStateRegistry); completedCheckpoints.add(completedCheckpoint); } } catch (Exception e) { 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 139f4846fa148..2e5f3d17767d5 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 @@ -61,6 +61,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.query.KvStateLocationRegistry; +import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.util.SerializedThrowable; @@ -69,8 +70,8 @@ import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; - import org.apache.flink.util.StringUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -456,7 +457,8 @@ public void enableCheckpointing( checkpointIDCounter, checkpointStore, checkpointDir, - ioExecutor); + ioExecutor, + SharedStateRegistry.DEFAULT_FACTORY); // register the master hooks on the checkpoint coordinator for (MasterTriggerRestoreHook hook : masterHooks) { 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 5ee7a9f97aa6c..e6d49d261def3 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 @@ -26,6 +26,7 @@ import org.apache.flink.api.common.accumulators.AccumulatorHelper; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.core.io.InputSplitSource; @@ -39,7 +40,6 @@ import org.apache.flink.runtime.jobgraph.JobEdge; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; -import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; 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 25df19b51aaed..d6019dbd317eb 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 @@ -89,9 +89,9 @@ import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedThrowable; import org.slf4j.Logger; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java index 0085890df28f0..0268b102be885 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandle.java @@ -65,27 +65,27 @@ public class IncrementalKeyedStateHandle implements KeyedStateHandle { private final UUID backendIdentifier; /** - * The key-group range covered by this state handle + * The key-group range covered by this state handle. */ private final KeyGroupRange keyGroupRange; /** - * The checkpoint Id + * The checkpoint Id. */ private final long checkpointId; /** - * Shared state in the incremental checkpoint. This i + * Shared state in the incremental checkpoint. */ private final Map sharedState; /** - * Private state in the incremental checkpoint + * Private state in the incremental checkpoint. */ private final Map privateState; /** - * Primary meta data state of the incremental checkpoint + * Primary meta data state of the incremental checkpoint. */ private final StreamStateHandle metaStateHandle; @@ -143,16 +143,21 @@ public UUID getBackendIdentifier() { @Override public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { - if (this.keyGroupRange.getIntersection(keyGroupRange) != KeyGroupRange.EMPTY_KEY_GROUP_RANGE) { - return this; - } else { - return null; - } + return KeyGroupRange.EMPTY_KEY_GROUP_RANGE.equals(this.keyGroupRange.getIntersection(keyGroupRange)) ? + null : this; } @Override public void discardState() throws Exception { + SharedStateRegistry registry = this.sharedStateRegistry; + final boolean isRegistered = (registry != null); + + LOG.trace("Discarding IncrementalKeyedStateHandle (registered = {}) for checkpoint {} from backend with id {}.", + isRegistered, + checkpointId, + backendIdentifier); + try { metaStateHandle.discardState(); } catch (Exception e) { @@ -168,19 +173,20 @@ public void discardState() throws Exception { // If this was not registered, we can delete the shared state. We can simply apply this // to all handles, because all handles that have not been created for the first time for this // are only placeholders at this point (disposing them is a NOP). - if (sharedStateRegistry == null) { - try { - StateUtil.bestEffortDiscardAllStateObjects(sharedState.values()); - } catch (Exception e) { - LOG.warn("Could not properly discard new sst file states.", e); - } - } else { + if (isRegistered) { // If this was registered, we only unregister all our referenced shared states // from the registry. for (StateHandleID stateHandleID : sharedState.keySet()) { - sharedStateRegistry.unregisterReference( + registry.unregisterReference( createSharedStateRegistryKeyFromFileName(stateHandleID)); } + } else { + // Otherwise, we assume to own those handles and dispose them directly. + try { + StateUtil.bestEffortDiscardAllStateObjects(sharedState.values()); + } catch (Exception e) { + LOG.warn("Could not properly discard new sst file states.", e); + } } } @@ -202,10 +208,21 @@ public long getStateSize() { @Override public void registerSharedStates(SharedStateRegistry stateRegistry) { - Preconditions.checkState(sharedStateRegistry == null, "The state handle has already registered its shared states."); + // This is a quick check to avoid that we register twice with the same registry. However, the code allows to + // register again with a different registry. The implication is that ownership is transferred to this new + // registry. This should only happen in case of a restart, when the CheckpointCoordinator creates a new + // SharedStateRegistry for the current attempt and the old registry becomes meaningless. We also assume that + // an old registry object from a previous run is due to be GCed and will never be used for registration again. + Preconditions.checkState( + sharedStateRegistry != stateRegistry, + "The state handle has already registered its shared states to the given registry."); sharedStateRegistry = Preconditions.checkNotNull(stateRegistry); + LOG.trace("Registering IncrementalKeyedStateHandle for checkpoint {} from backend with id {}.", + checkpointId, + backendIdentifier); + for (Map.Entry sharedStateHandle : sharedState.entrySet()) { SharedStateRegistryKey registryKey = createSharedStateRegistryKeyFromFileName(sharedStateHandle.getKey()); @@ -284,5 +301,18 @@ public int hashCode() { result = 31 * result + getMetaStateHandle().hashCode(); return result; } + + @Override + public String toString() { + return "IncrementalKeyedStateHandle{" + + "backendIdentifier=" + backendIdentifier + + ", keyGroupRange=" + keyGroupRange + + ", checkpointId=" + checkpointId + + ", sharedState=" + sharedState + + ", privateState=" + privateState + + ", metaStateHandle=" + metaStateHandle + + ", registered=" + (sharedStateRegistry != null) + + '}'; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java index 8e38ad4750d86..8092f6c72de8c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupsStateHandle.java @@ -141,7 +141,7 @@ public int hashCode() { public String toString() { return "KeyGroupsStateHandle{" + "groupRangeOffsets=" + groupRangeOffsets + - ", data=" + stateHandle + + ", stateHandle=" + stateHandle + '}'; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/MultiStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/MultiStreamStateHandle.java index b95daceece56a..1960c1c95f431 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/MultiStreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/MultiStreamStateHandle.java @@ -38,7 +38,7 @@ public class MultiStreamStateHandle implements StreamStateHandle { private final List stateHandles; private final long stateSize; - public MultiStreamStateHandle(List stateHandles) throws IOException { + public MultiStreamStateHandle(List stateHandles) { this.stateHandles = Preconditions.checkNotNull(stateHandles); long calculateSize = 0L; for(StreamStateHandle stateHandle : stateHandles) { @@ -62,6 +62,14 @@ public long getStateSize() { return stateSize; } + @Override + public String toString() { + return "MultiStreamStateHandle{" + + "stateHandles=" + stateHandles + + ", stateSize=" + stateSize + + '}'; + } + static final class MultiFSDataInputStream extends AbstractMultiFSDataInputStream { private final TreeMap stateHandleMap; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java index e0ca873668423..347f30c63ecbc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java @@ -38,13 +38,24 @@ * maintain the reference count of {@link StreamStateHandle}s by a key that (logically) identifies * them. */ -public class SharedStateRegistry { +public class SharedStateRegistry implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(SharedStateRegistry.class); + /** A singleton object for the default implementation of a {@link SharedStateRegistryFactory} */ + public static final SharedStateRegistryFactory DEFAULT_FACTORY = new SharedStateRegistryFactory() { + @Override + public SharedStateRegistry create(Executor deleteExecutor) { + return new SharedStateRegistry(deleteExecutor); + } + }; + /** All registered state objects by an artificial key */ private final Map registeredStates; + /** This flag indicates whether or not the registry is open or if close() was called */ + private boolean open; + /** Executor for async state deletion */ private final Executor asyncDisposalExecutor; @@ -56,6 +67,7 @@ public SharedStateRegistry() { public SharedStateRegistry(Executor asyncDisposalExecutor) { this.registeredStates = new HashMap<>(); this.asyncDisposalExecutor = Preconditions.checkNotNull(asyncDisposalExecutor); + this.open = true; } /** @@ -82,6 +94,9 @@ public Result registerReference(SharedStateRegistryKey registrationKey, StreamSt SharedStateRegistry.SharedStateEntry entry; synchronized (registeredStates) { + + Preconditions.checkState(open, "Attempt to register state to closed SharedStateRegistry."); + entry = registeredStates.get(registrationKey); if (entry == null) { @@ -96,6 +111,11 @@ public Result registerReference(SharedStateRegistryKey registrationKey, StreamSt // delete if this is a real duplicate if (!Objects.equals(state, entry.stateHandle)) { scheduledStateDeletion = state; + LOG.trace("Identified duplicate state registration under key {}. New state {} was determined to " + + "be an unnecessary copy of existing state {} and will be dropped.", + registrationKey, + state, + entry.stateHandle); } entry.increaseReferenceCount(); } @@ -112,7 +132,8 @@ public Result registerReference(SharedStateRegistryKey registrationKey, StreamSt * * @param registrationKey the shared state for which we release a reference. * @return the result of the request, consisting of the reference count after this operation - * and the state handle, or null if the state handle was deleted through this request. + * and the state handle, or null if the state handle was deleted through this request. Returns null if the registry + * was previously closed. */ public Result unregisterReference(SharedStateRegistryKey registrationKey) { @@ -123,6 +144,7 @@ public Result unregisterReference(SharedStateRegistryKey registrationKey) { SharedStateRegistry.SharedStateEntry entry; synchronized (registeredStates) { + entry = registeredStates.get(registrationKey); Preconditions.checkState(entry != null, @@ -164,10 +186,18 @@ public void registerAll(Iterable stateHandles) { } } + @Override + public String toString() { + synchronized (registeredStates) { + return "SharedStateRegistry{" + + "registeredStates=" + registeredStates + + '}'; + } + } + private void scheduleAsyncDelete(StreamStateHandle streamStateHandle) { // We do the small optimization to not issue discards for placeholders, which are NOPs. if (streamStateHandle != null && !isPlaceholder(streamStateHandle)) { - LOG.trace("Scheduled delete of state handle {}.", streamStateHandle); asyncDisposalExecutor.execute( new SharedStateRegistry.AsyncDisposalRunnable(streamStateHandle)); @@ -178,6 +208,13 @@ private boolean isPlaceholder(StreamStateHandle stateHandle) { return stateHandle instanceof PlaceholderStreamStateHandle; } + @Override + public void close() { + synchronized (registeredStates) { + open = false; + } + } + /** * An entry in the registry, tracking the handle and the corresponding reference count. */ @@ -279,13 +316,4 @@ public void run() { } } } - - /** - * Clears the registry. - */ - public void clear() { - synchronized (registeredStates) { - registeredStates.clear(); - } - } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java new file mode 100644 index 0000000000000..05c98258934ea --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java @@ -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. + */ + +package org.apache.flink.runtime.state; + +import java.util.concurrent.Executor; + +/** + * Simple factory to produce {@link SharedStateRegistry} objects. + */ +public interface SharedStateRegistryFactory { + + /** + * Factory method for {@link SharedStateRegistry}. + * + * @param deleteExecutor executor used to run (async) deletes. + * @return a SharedStateRegistry object + */ + SharedStateRegistry create(Executor deleteExecutor); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/ByteStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/ByteStreamStateHandle.java index 9ba9d35ff8939..3a43d4ffb979b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/ByteStreamStateHandle.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/ByteStreamStateHandle.java @@ -95,6 +95,7 @@ public int hashCode() { public String toString() { return "ByteStreamStateHandle{" + "handleName='" + handleName + '\'' + + ", dataBytes=" + data.length + '}'; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java index d293eea1c757e..edc29feb7c397 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorExternalizedCheckpointsTest.java @@ -18,14 +18,6 @@ package org.apache.flink.runtime.checkpoint; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -import java.io.File; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.savepoint.SavepointLoader; import org.apache.flink.runtime.concurrent.Executors; @@ -37,11 +29,22 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; +import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.filesystem.FileStateHandle; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + /** * CheckpointCoordinator tests for externalized checkpoints. * @@ -91,7 +94,8 @@ public void testTriggerAndConfirmSimpleExternalizedCheckpoint() new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), checkpointDir.getAbsolutePath(), - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index 88b95f5959fd7..26db772d52b29 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -79,7 +79,8 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { new StandaloneCheckpointIDCounter(), new FailingCompletedCheckpointStore(), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); coord.triggerCheckpoint(triggerTimestamp, false); @@ -111,7 +112,7 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { when(subtaskState.getSubtaskStateByOperatorID(OperatorID.fromJobVertexID(vertex.getJobvertexId()))).thenReturn(operatorSubtaskState); AcknowledgeCheckpoint acknowledgeMessage = new AcknowledgeCheckpoint(jid, executionAttemptId, checkpointId, new CheckpointMetrics(), subtaskState); - + try { coord.receiveAcknowledgeMessage(acknowledgeMessage); fail("Expected a checkpoint exception because the completed checkpoint store could not " + @@ -135,7 +136,7 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { private static final class FailingCompletedCheckpointStore implements CompletedCheckpointStore { @Override - public void recover(SharedStateRegistry sharedStateRegistry) throws Exception { + public void recover() throws Exception { throw new UnsupportedOperationException("Not implemented."); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index e23f6a2f7d8f7..2f860e0da2e6c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -28,9 +28,9 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; +import org.apache.flink.runtime.state.SharedStateRegistry; import org.junit.Test; - import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -46,14 +46,12 @@ import java.util.concurrent.Executor; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTest.mockExecutionVertex; - import static org.junit.Assert.assertArrayEquals; 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; - import static org.mockito.Matchers.eq; import static org.mockito.Matchers.isNull; import static org.mockito.Mockito.any; @@ -404,7 +402,8 @@ private static CheckpointCoordinator instantiateCheckpointCoordinator(JobID jid, new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(10), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); } private static T mockGeneric(Class clazz) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index d9af8797b7b17..45cbbc39d12b9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -36,32 +36,36 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint; import org.apache.flink.runtime.state.ChainedStateHandle; +import org.apache.flink.runtime.state.IncrementalKeyedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; import org.apache.flink.runtime.state.KeyGroupRangeOffsets; import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.PlaceholderStreamStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.SharedStateRegistryFactory; +import org.apache.flink.runtime.state.StateHandleID; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.filesystem.FileStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.RecoverableCompletedCheckpointStore; import org.apache.flink.runtime.util.TestByteStreamStateHandleDeepCompare; +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; -import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.mockito.verification.VerificationMode; import java.io.IOException; import java.io.Serializable; @@ -139,7 +143,8 @@ public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -199,7 +204,8 @@ public void testCheckpointAbortsIfTriggerTasksAreFinished() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -250,7 +256,8 @@ public void testCheckpointAbortsIfAckTasksAreNotExecuted() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // nothing should be happening assertEquals(0, coord.getNumberOfPendingCheckpoints()); @@ -302,7 +309,8 @@ public void testTriggerAndDeclineCheckpointSimple() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -406,7 +414,8 @@ public void testTriggerAndDeclineCheckpointComplex() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -525,7 +534,8 @@ public void testTriggerAndConfirmSimpleCheckpoint() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -692,7 +702,8 @@ public void testMultipleConcurrentCheckpoints() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -822,7 +833,8 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(10), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -986,7 +998,8 @@ public void testCheckpointTimeoutIsolated() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // trigger a checkpoint, partially acknowledged assertTrue(coord.triggerCheckpoint(timestamp, false)); @@ -1063,7 +1076,8 @@ public void testHandleMessagesForNonExistingCheckpoints() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertTrue(coord.triggerCheckpoint(timestamp, false)); @@ -1126,7 +1140,8 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertTrue(coord.triggerCheckpoint(timestamp, false)); @@ -1258,7 +1273,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); coord.startCheckpointScheduler(); @@ -1350,7 +1366,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), "dummy-path", - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); try { coord.startCheckpointScheduler(); @@ -1423,7 +1440,8 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertEquals(0, coord.getNumberOfPendingCheckpoints()); assertEquals(0, coord.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1574,7 +1592,8 @@ public void testSavepointsAreNotSubsumed() throws Exception { counter, new StandaloneCompletedCheckpointStore(10), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); String savepointDir = tmpFolder.newFolder().getAbsolutePath(); @@ -1680,7 +1699,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); coord.startCheckpointScheduler(); @@ -1753,7 +1773,8 @@ public void testMaxConcurrentAttempsWithSubsumption() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); coord.startCheckpointScheduler(); @@ -1835,7 +1856,8 @@ public ExecutionState answer(InvocationOnMock invocation){ new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); coord.startCheckpointScheduler(); @@ -1887,7 +1909,8 @@ public void testConcurrentSavepoints() throws Exception { checkpointIDCounter, new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); List> savepointFutures = new ArrayList<>(); @@ -1940,7 +1963,8 @@ public void testMinDelayBetweenSavepoints() throws Exception { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(2), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); String savepointDir = tmpFolder.newFolder().getAbsolutePath(); @@ -2002,7 +2026,8 @@ public void testRestoreLatestCheckpointedState() throws Exception { new StandaloneCheckpointIDCounter(), store, null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); @@ -2116,7 +2141,8 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); @@ -2237,7 +2263,8 @@ public void testRestoreLatestCheckpointFailureWhenParallelismChanges() throws Ex new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); @@ -2395,7 +2422,8 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // trigger the checkpoint coord.triggerCheckpoint(timestamp, false); @@ -2686,7 +2714,8 @@ public void testStateRecoveryWithTopologyChange(int scaleType) throws Exception new StandaloneCheckpointIDCounter(), standaloneCompletedCheckpointStore, null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); coord.restoreLatestCheckpointedState(tasks, false, true); @@ -2847,7 +2876,8 @@ public void testExternalizedCheckpoints() throws Exception { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), "fake-directory", - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); assertTrue(coord.triggerCheckpoint(timestamp, false)); @@ -3351,7 +3381,8 @@ public void testStopPeriodicScheduler() throws Exception { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // Periodic CheckpointTriggerResult triggerResult = coord.triggerCheckpoint( @@ -3529,7 +3560,8 @@ public void testCheckpointStatsTrackerPendingCheckpointCallback() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); CheckpointStatsTracker tracker = mock(CheckpointStatsTracker.class); coord.setCheckpointStatsTracker(tracker); @@ -3567,7 +3599,8 @@ public void testCheckpointStatsTrackerRestoreCallback() throws Exception { new StandaloneCheckpointIDCounter(), store, null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); store.addCheckpoint(new CompletedCheckpoint( new JobID(), @@ -3623,7 +3656,8 @@ public void testSavepointsAreNotAddedToCompletedCheckpointStore() throws Excepti checkpointIDCounter, completedCheckpointStore, null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // trigger a first checkpoint assertTrue( @@ -3673,4 +3707,245 @@ public void testSavepointsAreNotAddedToCompletedCheckpointStore() throws Excepti "The latest completed (proper) checkpoint should have been added to the completed checkpoint store.", completedCheckpointStore.getLatestCheckpoint().getCheckpointID() == checkpointIDCounter.getLast()); } + + @Test + public void testSharedStateRegistrationOnRestore() throws Exception { + + final JobID jid = new JobID(); + final long timestamp = System.currentTimeMillis(); + + final JobVertexID jobVertexID1 = new JobVertexID(); + + int parallelism1 = 2; + int maxParallelism1 = 4; + + final ExecutionJobVertex jobVertex1 = mockExecutionJobVertex( + jobVertexID1, + parallelism1, + maxParallelism1); + + List allExecutionVertices = new ArrayList<>(parallelism1); + + allExecutionVertices.addAll(Arrays.asList(jobVertex1.getTaskVertices())); + + ExecutionVertex[] arrayExecutionVertices = + allExecutionVertices.toArray(new ExecutionVertex[allExecutionVertices.size()]); + + RecoverableCompletedCheckpointStore store = new RecoverableCompletedCheckpointStore(10); + + final List createdSharedStateRegistries = new ArrayList<>(2); + + // set up the coordinator and validate the initial state + CheckpointCoordinator coord = new CheckpointCoordinator( + jid, + 600000, + 600000, + 0, + Integer.MAX_VALUE, + ExternalizedCheckpointSettings.none(), + arrayExecutionVertices, + arrayExecutionVertices, + arrayExecutionVertices, + new StandaloneCheckpointIDCounter(), + store, + null, + Executors.directExecutor(), + new SharedStateRegistryFactory() { + @Override + public SharedStateRegistry create(Executor deleteExecutor) { + SharedStateRegistry instance = new SharedStateRegistry(deleteExecutor); + createdSharedStateRegistries.add(instance); + return instance; + } + }); + + final int numCheckpoints = 3; + + List keyGroupPartitions1 = + StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); + + for (int i = 0; i < numCheckpoints; ++i) { + performIncrementalCheckpoint(jid, coord, jobVertex1, keyGroupPartitions1, timestamp + i, i); + } + + List completedCheckpoints = coord.getSuccessfulCheckpoints(); + assertEquals(numCheckpoints, completedCheckpoints.size()); + + int sharedHandleCount = 0; + + List> sharedHandlesByCheckpoint = new ArrayList<>(numCheckpoints); + + for (int i = 0; i < numCheckpoints; ++i) { + sharedHandlesByCheckpoint.add(new HashMap(2)); + } + + int cp = 0; + for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { + for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { + for (OperatorSubtaskState subtaskState : taskState.getStates()) { + for (KeyedStateHandle keyedStateHandle : subtaskState.getManagedKeyedState()) { + // test we are once registered with the current registry + verify(keyedStateHandle, times(1)).registerSharedStates(createdSharedStateRegistries.get(0)); + IncrementalKeyedStateHandle incrementalKeyedStateHandle = (IncrementalKeyedStateHandle) keyedStateHandle; + + sharedHandlesByCheckpoint.get(cp).putAll(incrementalKeyedStateHandle.getSharedState()); + + for (StreamStateHandle streamStateHandle : incrementalKeyedStateHandle.getSharedState().values()) { + assertTrue(!(streamStateHandle instanceof PlaceholderStreamStateHandle)); + verify(streamStateHandle, never()).discardState(); + ++sharedHandleCount; + } + + for (StreamStateHandle streamStateHandle : incrementalKeyedStateHandle.getPrivateState().values()) { + verify(streamStateHandle, never()).discardState(); + } + + verify(incrementalKeyedStateHandle.getMetaStateHandle(), never()).discardState(); + } + + verify(subtaskState, never()).discardState(); + } + } + ++cp; + } + + // 2 (parallelism) x (1 (CP0) + 2 (CP1) + 2 (CP2)) = 10 + assertEquals(10, sharedHandleCount); + + // discard CP0 + store.removeOldestCheckpoint(); + + // we expect no shared state was discarded because the state of CP0 is still referenced by CP1 + for (Map cpList : sharedHandlesByCheckpoint) { + for (StreamStateHandle streamStateHandle : cpList.values()) { + verify(streamStateHandle, never()).discardState(); + } + } + + // shutdown the store + store.shutdown(JobStatus.SUSPENDED); + + // restore the store + Map tasks = new HashMap<>(); + tasks.put(jobVertexID1, jobVertex1); + coord.restoreLatestCheckpointedState(tasks, true, false); + + // validate that all shared states are registered again after the recovery. + cp = 0; + for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { + for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { + for (OperatorSubtaskState subtaskState : taskState.getStates()) { + for (KeyedStateHandle keyedStateHandle : subtaskState.getManagedKeyedState()) { + VerificationMode verificationMode; + // test we are once registered with the new registry + if (cp > 0) { + verificationMode = times(1); + } else { + verificationMode = never(); + } + + //check that all are registered with the new registry + verify(keyedStateHandle, verificationMode).registerSharedStates(createdSharedStateRegistries.get(1)); + } + } + } + ++cp; + } + + // discard CP1 + store.removeOldestCheckpoint(); + + // we expect that all shared state from CP0 is no longer referenced and discarded. CP2 is still live and also + // references the state from CP1, so we expect they are not discarded. + for (Map cpList : sharedHandlesByCheckpoint) { + for (Map.Entry entry : cpList.entrySet()) { + String key = entry.getKey().getKeyString(); + int belongToCP = Integer.parseInt(String.valueOf(key.charAt(key.length() - 1))); + if (belongToCP == 0) { + verify(entry.getValue(), times(1)).discardState(); + } else { + verify(entry.getValue(), never()).discardState(); + } + } + } + + // discard CP2 + store.removeOldestCheckpoint(); + + // we expect all shared state was discarded now, because all CPs are + for (Map cpList : sharedHandlesByCheckpoint) { + for (StreamStateHandle streamStateHandle : cpList.values()) { + verify(streamStateHandle, times(1)).discardState(); + } + } + } + + private void performIncrementalCheckpoint( + JobID jid, + CheckpointCoordinator coord, + ExecutionJobVertex jobVertex1, + List keyGroupPartitions1, + long timestamp, + int cpSequenceNumber) throws Exception { + + // trigger the checkpoint + coord.triggerCheckpoint(timestamp, false); + + assertTrue(coord.getPendingCheckpoints().keySet().size() == 1); + long checkpointId = Iterables.getOnlyElement(coord.getPendingCheckpoints().keySet()); + + for (int index = 0; index < jobVertex1.getParallelism(); index++) { + + KeyGroupRange keyGroupRange = keyGroupPartitions1.get(index); + + Map privateState = new HashMap<>(); + privateState.put( + new StateHandleID("private-1"), + spy(new ByteStreamStateHandle("private-1", new byte[]{'p'}))); + + Map sharedState = new HashMap<>(); + + // let all but the first CP overlap by one shared state. + if (cpSequenceNumber > 0) { + sharedState.put( + new StateHandleID("shared-" + (cpSequenceNumber - 1)), + spy(new PlaceholderStreamStateHandle())); + } + + sharedState.put( + new StateHandleID("shared-" + cpSequenceNumber), + spy(new ByteStreamStateHandle("shared-" + cpSequenceNumber + "-" + keyGroupRange, new byte[]{'s'}))); + + IncrementalKeyedStateHandle managedState = + spy(new IncrementalKeyedStateHandle( + new UUID(42L, 42L), + keyGroupRange, + checkpointId, + sharedState, + privateState, + spy(new ByteStreamStateHandle("meta", new byte[]{'m'})))); + + OperatorSubtaskState operatorSubtaskState = + spy(new OperatorSubtaskState(null, + Collections.emptyList(), + Collections.emptyList(), + Collections.singletonList(managedState), + Collections.emptyList())); + + Map opStates = new HashMap<>(); + + opStates.put(jobVertex1.getOperatorIDs().get(0), operatorSubtaskState); + + TaskStateSnapshot taskStateSnapshot = new TaskStateSnapshot(opStates); + + AcknowledgeCheckpoint acknowledgeCheckpoint = new AcknowledgeCheckpoint( + jid, + jobVertex1.getTaskVertices()[index].getCurrentExecutionAttempt().getAttemptId(), + checkpointId, + new CheckpointMetrics(), + taskStateSnapshot); + + coord.receiveAcknowledgeMessage(acknowledgeCheckpoint); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index 6ce071b2269bd..791bffa7635ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.OperatorStateHandle; +import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.util.SerializableObject; @@ -109,7 +110,8 @@ public void testSetState() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); // create ourselves a checkpoint with state final long timestamp = 34623786L; @@ -186,7 +188,8 @@ public void testNoCheckpointAvailable() { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); try { coord.restoreLatestCheckpointedState(new HashMap(), true, false); @@ -243,7 +246,8 @@ public void testNonRestoredState() throws Exception { new StandaloneCheckpointIDCounter(), new StandaloneCompletedCheckpointStore(1), null, - Executors.directExecutor()); + Executors.directExecutor(), + SharedStateRegistry.DEFAULT_FACTORY); StreamStateHandle serializedState = CheckpointCoordinatorTest .generateChainedStateHandle(new SerializableObject()) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java index 77423c213fd81..dc2b11ebd08bc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java @@ -18,13 +18,14 @@ package org.apache.flink.runtime.checkpoint; -import org.apache.curator.framework.CuratorFramework; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.jobgraph.JobStatus; import org.apache.flink.runtime.state.RetrievableStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper; import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; + +import org.apache.curator.framework.CuratorFramework; import org.apache.zookeeper.data.Stat; import org.junit.AfterClass; import org.junit.Before; @@ -106,8 +107,9 @@ public void testRecover() throws Exception { assertEquals(3, checkpoints.getNumberOfRetainedCheckpoints()); // Recover - sharedStateRegistry.clear(); - checkpoints.recover(sharedStateRegistry); + sharedStateRegistry.close(); + sharedStateRegistry = new SharedStateRegistry(); + checkpoints.recover(); assertEquals(3, ZOOKEEPER.getClient().getChildren().forPath(CHECKPOINT_PATH).size()); assertEquals(3, checkpoints.getNumberOfRetainedCheckpoints()); @@ -148,8 +150,8 @@ public void testShutdownDiscardsCheckpoints() throws Exception { assertEquals(0, store.getNumberOfRetainedCheckpoints()); assertNull(client.checkExists().forPath(CHECKPOINT_PATH + ZooKeeperCompletedCheckpointStore.checkpointIdToPath(checkpoint.getCheckpointID()))); - sharedStateRegistry.clear(); - store.recover(sharedStateRegistry); + sharedStateRegistry.close(); + store.recover(); assertEquals(0, store.getNumberOfRetainedCheckpoints()); } @@ -182,8 +184,8 @@ public void testSuspendKeepsCheckpoints() throws Exception { assertEquals("The checkpoint node should not be locked.", 0, stat.getNumChildren()); // Recover again - sharedStateRegistry.clear(); - store.recover(sharedStateRegistry); + sharedStateRegistry.close(); + store.recover(); CompletedCheckpoint recovered = store.getLatestCheckpoint(); assertEquals(checkpoint, recovered); @@ -209,8 +211,8 @@ public void testLatestCheckpointRecovery() throws Exception { checkpointStore.addCheckpoint(checkpoint); } - sharedStateRegistry.clear(); - checkpointStore.recover(sharedStateRegistry); + sharedStateRegistry.close(); + checkpointStore.recover(); CompletedCheckpoint latestCheckpoint = checkpointStore.getLatestCheckpoint(); @@ -239,8 +241,9 @@ public void testConcurrentCheckpointOperations() throws Exception { zkCheckpointStore1.addCheckpoint(completedCheckpoint); // recover the checkpoint by a different checkpoint store - sharedStateRegistry.clear(); - zkCheckpointStore2.recover(sharedStateRegistry); + sharedStateRegistry.close(); + sharedStateRegistry = new SharedStateRegistry(); + zkCheckpointStore2.recover(); CompletedCheckpoint recoveredCheckpoint = zkCheckpointStore2.getLatestCheckpoint(); assertTrue(recoveredCheckpoint instanceof TestCompletedCheckpoint); 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 91bab85fef84a..3171f1f84e388 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 @@ -52,7 +52,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyCollection; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -162,11 +161,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { stateStorage, Executors.directExecutor()); - SharedStateRegistry sharedStateRegistry = spy(new SharedStateRegistry()); - zooKeeperCompletedCheckpointStore.recover(sharedStateRegistry); - - verify(retrievableStateHandle1.retrieveState(), times(1)).registerSharedStatesAfterRestored(sharedStateRegistry); - verify(retrievableStateHandle2.retrieveState(), times(1)).registerSharedStatesAfterRestored(sharedStateRegistry); + zooKeeperCompletedCheckpointStore.recover(); CompletedCheckpoint latestCompletedCheckpoint = zooKeeperCompletedCheckpointStore.getLatestCheckpoint(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandleTest.java index c1b3ccdfa34e1..9f6f88ec42eb8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/IncrementalKeyedStateHandleTest.java @@ -19,12 +19,15 @@ package org.apache.flink.runtime.state; import org.apache.flink.runtime.checkpoint.savepoint.CheckpointTestUtils; + import org.junit.Test; import java.util.Map; import java.util.Random; import java.util.UUID; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.powermock.api.mockito.PowerMockito.spy; @@ -59,8 +62,6 @@ public void testUnregisteredDiscarding() throws Exception { @Test public void testSharedStateDeRegistration() throws Exception { - Random rnd = new Random(42); - SharedStateRegistry registry = spy(new SharedStateRegistry()); // Create two state handles with overlapping shared state @@ -186,6 +187,76 @@ public void testSharedStateDeRegistration() throws Exception { verify(stateHandle2.getMetaStateHandle(), times(1)).discardState(); } + /** + * This tests that re-registration of shared state with another registry works as expected. This simulates a + * recovery from a checkpoint, when the checkpoint coordinator creates a new shared state registry and re-registers + * all live checkpoint states. + */ + @Test + public void testSharedStateReRegistration() throws Exception { + + SharedStateRegistry stateRegistryA = spy(new SharedStateRegistry()); + + IncrementalKeyedStateHandle stateHandleX = create(new Random(1)); + IncrementalKeyedStateHandle stateHandleY = create(new Random(2)); + IncrementalKeyedStateHandle stateHandleZ = create(new Random(3)); + + // Now we register first time ... + stateHandleX.registerSharedStates(stateRegistryA); + stateHandleY.registerSharedStates(stateRegistryA); + stateHandleZ.registerSharedStates(stateRegistryA); + + try { + // Second attempt should fail + stateHandleX.registerSharedStates(stateRegistryA); + fail("Should not be able to register twice with the same registry."); + } catch (IllegalStateException ignore) { + } + + // Everything should be discarded for this handle + stateHandleZ.discardState(); + verify(stateHandleZ.getMetaStateHandle(), times(1)).discardState(); + for (StreamStateHandle stateHandle : stateHandleZ.getSharedState().values()) { + verify(stateHandle, times(1)).discardState(); + } + + // Close the first registry + stateRegistryA.close(); + + // Attempt to register to closed registry should trigger exception + try { + create(new Random(4)).registerSharedStates(stateRegistryA); + fail("Should not be able to register new state to closed registry."); + } catch (IllegalStateException ignore) { + } + + // All state should still get discarded + stateHandleY.discardState(); + verify(stateHandleY.getMetaStateHandle(), times(1)).discardState(); + for (StreamStateHandle stateHandle : stateHandleY.getSharedState().values()) { + verify(stateHandle, times(1)).discardState(); + } + + // This should still be unaffected + verify(stateHandleX.getMetaStateHandle(), never()).discardState(); + for (StreamStateHandle stateHandle : stateHandleX.getSharedState().values()) { + verify(stateHandle, never()).discardState(); + } + + // We re-register the handle with a new registry + SharedStateRegistry sharedStateRegistryB = spy(new SharedStateRegistry()); + stateHandleX.registerSharedStates(sharedStateRegistryB); + stateHandleX.discardState(); + + // Should be completely discarded because it is tracked through the new registry + verify(stateHandleX.getMetaStateHandle(), times(1)).discardState(); + for (StreamStateHandle stateHandle : stateHandleX.getSharedState().values()) { + verify(stateHandle, times(1)).discardState(); + } + + sharedStateRegistryB.close(); + } + private static IncrementalKeyedStateHandle create(Random rnd) { return new IncrementalKeyedStateHandle( UUID.nameUUIDFromBytes("test".getBytes()), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/RecoverableCompletedCheckpointStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/RecoverableCompletedCheckpointStore.java index a0c441247ae56..037ecd17cf268 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/RecoverableCompletedCheckpointStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/RecoverableCompletedCheckpointStore.java @@ -21,7 +21,8 @@ import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; import org.apache.flink.runtime.jobgraph.JobStatus; -import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,14 +42,21 @@ public class RecoverableCompletedCheckpointStore implements CompletedCheckpointS private final ArrayDeque suspended = new ArrayDeque<>(2); + private final int maxRetainedCheckpoints; + + public RecoverableCompletedCheckpointStore() { + this(1); + } + + public RecoverableCompletedCheckpointStore(int maxRetainedCheckpoints) { + Preconditions.checkArgument(maxRetainedCheckpoints > 0); + this.maxRetainedCheckpoints = maxRetainedCheckpoints; + } + @Override - public void recover(SharedStateRegistry sharedStateRegistry) throws Exception { + public void recover() throws Exception { checkpoints.addAll(suspended); suspended.clear(); - - for (CompletedCheckpoint checkpoint : checkpoints) { - checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry); - } } @Override @@ -56,13 +64,16 @@ public void addCheckpoint(CompletedCheckpoint checkpoint) throws Exception { checkpoints.addLast(checkpoint); - - if (checkpoints.size() > 1) { - CompletedCheckpoint checkpointToSubsume = checkpoints.removeFirst(); - checkpointToSubsume.discardOnSubsume(); + if (checkpoints.size() > maxRetainedCheckpoints) { + removeOldestCheckpoint(); } } + public void removeOldestCheckpoint() throws Exception { + CompletedCheckpoint checkpointToSubsume = checkpoints.removeFirst(); + checkpointToSubsume.discardOnSubsume(); + } + @Override public CompletedCheckpoint getLatestCheckpoint() throws Exception { return checkpoints.isEmpty() ? null : checkpoints.getLast(); @@ -96,7 +107,7 @@ public int getNumberOfRetainedCheckpoints() { @Override public int getMaxNumberOfRetainedCheckpoints() { - return 1; + return maxRetainedCheckpoints; } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index cb8639b0dbe60..1ba5fb12bbdab 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.runtime.tasks; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.fs.CloseableRegistry; @@ -779,9 +780,11 @@ public CheckpointStreamFactory createSavepointStreamFactory(StreamOperator op } private String createOperatorIdentifier(StreamOperator operator, int vertexId) { + + TaskInfo taskInfo = getEnvironment().getTaskInfo(); return operator.getClass().getSimpleName() + - "_" + vertexId + - "_" + getEnvironment().getTaskInfo().getIndexOfThisSubtask(); + "_" + operator.getOperatorID() + + "_(" + taskInfo.getIndexOfThisSubtask() + "/" + taskInfo.getNumberOfParallelSubtasks() + ")"; } /** @@ -892,18 +895,22 @@ public void run() { if (asyncCheckpointState.compareAndSet(CheckpointingOperation.AsynCheckpointState.RUNNING, CheckpointingOperation.AsynCheckpointState.COMPLETED)) { + TaskStateSnapshot acknowledgedState = hasState ? taskOperatorSubtaskStates : null; + // we signal stateless tasks by reporting null, so that there are no attempts to assign empty state // to stateless tasks on restore. This enables simple job modifications that only concern // stateless without the need to assign them uids to match their (always empty) states. owner.getEnvironment().acknowledgeCheckpoint( checkpointMetaData.getCheckpointId(), checkpointMetrics, - hasState ? taskOperatorSubtaskStates : null); + acknowledgedState); + + LOG.debug("{} - finished asynchronous part of checkpoint {}. Asynchronous duration: {} ms", + owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); + + LOG.trace("{} - reported the following states in snapshot for checkpoint {}: {}.", + owner.getName(), checkpointMetaData.getCheckpointId(), acknowledgedState); - if (LOG.isDebugEnabled()) { - LOG.debug("{} - finished asynchronous part of checkpoint {}. Asynchronous duration: {} ms", - owner.getName(), checkpointMetaData.getCheckpointId(), asyncDurationMillis); - } } else { LOG.debug("{} - asynchronous part of checkpoint {} could not be completed because it was closed before.", owner.getName(), 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 22ed84723e552..c525a37d9aa93 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 @@ -27,6 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple4; 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.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.core.fs.Path; @@ -48,21 +49,22 @@ import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; +import org.apache.curator.test.TestingServer; import org.junit.After; -import org.junit.AfterClass; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; +import java.io.File; import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.List; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.flink.test.checkpointing.AbstractEventTimeWindowCheckpointingITCase.StateBackendEnum.ROCKSDB_INCREMENTAL_ZK; import static org.apache.flink.test.util.TestUtils.tryExecute; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -87,6 +89,8 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog private static TestStreamEnvironment env; + private static TestingServer zkServer; + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); @@ -101,11 +105,27 @@ public abstract class AbstractEventTimeWindowCheckpointingITCase extends TestLog } enum StateBackendEnum { - MEM, FILE, ROCKSDB_FULLY_ASYNC, ROCKSDB_INCREMENTAL, MEM_ASYNC, FILE_ASYNC + MEM, FILE, ROCKSDB_FULLY_ASYNC, ROCKSDB_INCREMENTAL, ROCKSDB_INCREMENTAL_ZK, MEM_ASYNC, FILE_ASYNC } - @BeforeClass - public static void startTestCluster() { + @Before + public void startTestCluster() throws Exception { + + // print a message when starting a test method to avoid Travis' "Maven produced no + // output for xxx seconds." messages + System.out.println( + "Starting " + getClass().getCanonicalName() + "#" + name.getMethodName() + "."); + + // Testing HA Scenario / ZKCompletedCheckpointStore with incremental checkpoints + if (ROCKSDB_INCREMENTAL_ZK.equals(stateBackendEnum)) { + zkServer = new TestingServer(); + zkServer.start(); + } + + TemporaryFolder temporaryFolder = new TemporaryFolder(); + temporaryFolder.create(); + final File haDir = temporaryFolder.newFolder(); + Configuration config = new Configuration(); config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2); config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2); @@ -113,28 +133,18 @@ public static void startTestCluster() { // the default network buffers size (10% of heap max =~ 150MB) seems to much for this test case config.setLong(TaskManagerOptions.NETWORK_BUFFERS_MEMORY_MAX, 80L << 20); // 80 MB + if (zkServer != null) { + config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER"); + config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkServer.getConnectString()); + config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haDir.toURI().toString()); + } + cluster = new LocalFlinkMiniCluster(config, false); cluster.start(); env = new TestStreamEnvironment(cluster, PARALLELISM); env.getConfig().setUseSnapshotCompression(true); - } - - @AfterClass - public static void stopTestCluster() { - if (cluster != null) { - cluster.stop(); - } - } - - @Before - public void beforeTest() throws IOException { - // print a message when starting a test method to avoid Travis' "Maven produced no - // output for xxx seconds." messages - System.out.println( - "Starting " + getClass().getCanonicalName() + "#" + name.getMethodName() + "."); - // init state back-end switch (stateBackendEnum) { case MEM: this.stateBackend = new MemoryStateBackend(MAX_MEM_STATE_SIZE, false); @@ -159,7 +169,8 @@ public void beforeTest() throws IOException { this.stateBackend = rdb; break; } - case ROCKSDB_INCREMENTAL: { + case ROCKSDB_INCREMENTAL: + case ROCKSDB_INCREMENTAL_ZK: { String rocksDb = tempFolder.newFolder().getAbsolutePath(); String backups = tempFolder.newFolder().getAbsolutePath(); // we use the fs backend with small threshold here to test the behaviour with file @@ -173,16 +184,25 @@ public void beforeTest() throws IOException { this.stateBackend = rdb; break; } - + default: + throw new IllegalStateException("No backend selected."); } } - /** - * Prints a message when finishing a test method to avoid Travis' "Maven produced no output - * for xxx seconds." messages. - */ @After - public void afterTest() { + public void stopTestCluster() throws IOException { + if (cluster != null) { + cluster.stop(); + cluster = null; + } + + if (zkServer != null) { + zkServer.stop(); + zkServer = null; + } + + //Prints a message when finishing a test method to avoid Travis' "Maven produced no output + // for xxx seconds." messages. System.out.println( "Finished " + getClass().getCanonicalName() + "#" + name.getMethodName() + "."); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/HAIncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/HAIncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.java new file mode 100644 index 0000000000000..394815f2ae61a --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/HAIncrementalRocksDbBackendEventTimeWindowCheckpointingITCase.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.test.checkpointing; + +/** + * Integration tests for incremental RocksDB backend. + */ +public class HAIncrementalRocksDbBackendEventTimeWindowCheckpointingITCase extends AbstractEventTimeWindowCheckpointingITCase { + + public HAIncrementalRocksDbBackendEventTimeWindowCheckpointingITCase() { + super(StateBackendEnum.ROCKSDB_INCREMENTAL_ZK); + } + + @Override + protected int numElementsPerKey() { + return 3000; + } + + @Override + protected int windowSize() { + return 1000; + } + + @Override + protected int windowSlide() { + return 100; + } + + @Override + protected int numKeys() { + return 100; + } +} From d29bed38311f7a01d2241fbf8fa26eac7f012f53 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 28 Jul 2017 15:01:35 +0200 Subject: [PATCH 022/129] [FLINK-7268] Add delaying executor in *EventTimeWindowCheckpointingITCase This helps tease out races, for example the recently discovered one in cleanup of incremental state handles at the SharedStateRegistry. (cherry picked from commit d7683cc) --- ...ractEventTimeWindowCheckpointingITCase.java | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) 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 c525a37d9aa93..4d5fa719c73ec 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 @@ -31,6 +31,8 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.CheckpointListener; @@ -62,6 +64,9 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.flink.test.checkpointing.AbstractEventTimeWindowCheckpointingITCase.StateBackendEnum.ROCKSDB_INCREMENTAL_ZK; @@ -139,7 +144,18 @@ public void startTestCluster() throws Exception { config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, haDir.toURI().toString()); } - cluster = new LocalFlinkMiniCluster(config, false); + // purposefully delay in the executor to tease out races + final ScheduledExecutorService executor = Executors.newScheduledThreadPool(10); + HighAvailabilityServices haServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices( + config, + new Executor() { + @Override + public void execute(Runnable command) { + executor.schedule(command, 500, MILLISECONDS); + } + }); + + cluster = new LocalFlinkMiniCluster(config, haServices, false); cluster.start(); env = new TestStreamEnvironment(cluster, PARALLELISM); From 843f0cbc105ba165c6dcf66b46dc0d0d120bbfb4 Mon Sep 17 00:00:00 2001 From: Stefan Richter Date: Mon, 7 Aug 2017 16:22:32 +0200 Subject: [PATCH 023/129] [FLINK-7362] [checkpoints] Savepoint property is lost after de/serialization of CheckpointProperties --- .../runtime/checkpoint/CheckpointProperties.java | 14 ++++++++++++-- .../checkpoint/CheckpointPropertiesTest.java | 8 ++++++++ .../checkpoint/CompletedCheckpointTest.java | 6 +++--- .../runtime/checkpoint/PendingCheckpointTest.java | 14 +++++++------- .../checkpoint/RestoredCheckpointStatsTest.java | 2 +- 5 files changed, 31 insertions(+), 13 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java index 6df7e71c0afc5..1233b6ec419fc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointProperties.java @@ -39,6 +39,7 @@ public class CheckpointProperties implements Serializable { private final boolean forced; private final boolean externalize; + private final boolean savepoint; private final boolean discardSubsumed; private final boolean discardFinished; @@ -49,6 +50,7 @@ public class CheckpointProperties implements Serializable { CheckpointProperties( boolean forced, boolean externalize, + boolean savepoint, boolean discardSubsumed, boolean discardFinished, boolean discardCancelled, @@ -57,6 +59,7 @@ public class CheckpointProperties implements Serializable { this.forced = forced; this.externalize = externalize; + this.savepoint = savepoint; this.discardSubsumed = discardSubsumed; this.discardFinished = discardFinished; this.discardCancelled = discardCancelled; @@ -183,7 +186,7 @@ boolean discardOnJobSuspended() { * @return true if the properties describe a savepoint, false otherwise. */ public boolean isSavepoint() { - return this == STANDARD_SAVEPOINT; + return savepoint; } // ------------------------------------------------------------------------ @@ -201,6 +204,7 @@ public boolean equals(Object o) { CheckpointProperties that = (CheckpointProperties) o; return forced == that.forced && externalize == that.externalize && + savepoint == that.savepoint && discardSubsumed == that.discardSubsumed && discardFinished == that.discardFinished && discardCancelled == that.discardCancelled && @@ -212,6 +216,7 @@ public boolean equals(Object o) { public int hashCode() { int result = (forced ? 1 : 0); result = 31 * result + (externalize ? 1 : 0); + result = 31 * result + (savepoint ? 1 : 0); result = 31 * result + (discardSubsumed ? 1 : 0); result = 31 * result + (discardFinished ? 1 : 0); result = 31 * result + (discardCancelled ? 1 : 0); @@ -224,7 +229,8 @@ public int hashCode() { public String toString() { return "CheckpointProperties{" + "forced=" + forced + - ", externalize=" + externalizeCheckpoint() + + ", externalized=" + externalizeCheckpoint() + + ", savepoint=" + savepoint + ", discardSubsumed=" + discardSubsumed + ", discardFinished=" + discardFinished + ", discardCancelled=" + discardCancelled + @@ -236,6 +242,7 @@ public String toString() { // ------------------------------------------------------------------------ private static final CheckpointProperties STANDARD_SAVEPOINT = new CheckpointProperties( + true, true, true, false, @@ -245,6 +252,7 @@ public String toString() { false); private static final CheckpointProperties STANDARD_CHECKPOINT = new CheckpointProperties( + false, false, false, true, @@ -256,6 +264,7 @@ public String toString() { private static final CheckpointProperties EXTERNALIZED_CHECKPOINT_RETAINED = new CheckpointProperties( false, true, + false, true, true, false, // Retain on cancellation @@ -265,6 +274,7 @@ public String toString() { private static final CheckpointProperties EXTERNALIZED_CHECKPOINT_DELETED = new CheckpointProperties( false, true, + false, true, true, true, // Delete on cancellation diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java index 52ac54ca37159..a0509c43ccb90 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointPropertiesTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.checkpoint; +import org.apache.flink.util.InstantiationUtil; + import org.junit.Test; import static org.junit.Assert.assertFalse; @@ -109,6 +111,12 @@ public void testIsSavepoint() throws Exception { { CheckpointProperties props = CheckpointProperties.forStandardSavepoint(); assertTrue(props.isSavepoint()); + + CheckpointProperties deserializedCheckpointProperties = + InstantiationUtil.deserializeObject( + InstantiationUtil.serializeObject(props), + getClass().getClassLoader()); + assertTrue(deserializedCheckpointProperties.isSavepoint()); } } 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 4846879244d14..293675c14e932 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 @@ -82,7 +82,7 @@ public void testCleanUpOnSubsume() throws Exception { operatorStates.put(new OperatorID(), state); boolean discardSubsumed = true; - CheckpointProperties props = new CheckpointProperties(false, false, discardSubsumed, true, true, true, true); + CheckpointProperties props = new CheckpointProperties(false, false, false, discardSubsumed, true, true, true, true); CompletedCheckpoint checkpoint = new CompletedCheckpoint( new JobID(), 0, 0, 1, @@ -122,7 +122,7 @@ public void testCleanUpOnShutdown() throws Exception { Mockito.reset(state); // Keep - CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false); + CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false, false); CompletedCheckpoint checkpoint = new CompletedCheckpoint( new JobID(), 0, 0, 1, new HashMap<>(operatorStates), @@ -139,7 +139,7 @@ public void testCleanUpOnShutdown() throws Exception { assertEquals(true, file.exists()); // Discard - props = new CheckpointProperties(false, false, true, true, true, true, true); + props = new CheckpointProperties(false, false, false, true, true, true, true, true); checkpoint = new CompletedCheckpoint( new JobID(), 0, 0, 1, new HashMap<>(operatorStates), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index 7ebb49a4c1c61..ef31f0a3aa63c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -80,7 +80,7 @@ public class PendingCheckpointTest { @Test public void testCanBeSubsumed() throws Exception { // Forced checkpoints cannot be subsumed - CheckpointProperties forced = new CheckpointProperties(true, true, false, false, false, false, false); + CheckpointProperties forced = new CheckpointProperties(true, true, false, false, false, false, false, false); PendingCheckpoint pending = createPendingCheckpoint(forced, "ignored"); assertFalse(pending.canBeSubsumed()); @@ -92,7 +92,7 @@ public void testCanBeSubsumed() throws Exception { } // Non-forced checkpoints can be subsumed - CheckpointProperties subsumed = new CheckpointProperties(false, true, false, false, false, false, false); + CheckpointProperties subsumed = new CheckpointProperties(false, true, false, false, false, false, false, false); pending = createPendingCheckpoint(subsumed, "ignored"); assertTrue(pending.canBeSubsumed()); } @@ -106,7 +106,7 @@ public void testPersistExternally() throws Exception { File tmp = tmpFolder.newFolder(); // Persisted checkpoint - CheckpointProperties persisted = new CheckpointProperties(false, true, false, false, false, false, false); + CheckpointProperties persisted = new CheckpointProperties(false, true, false, false, false, false, false, false); PendingCheckpoint pending = createPendingCheckpoint(persisted, tmp.getAbsolutePath()); pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); @@ -115,7 +115,7 @@ public void testPersistExternally() throws Exception { assertEquals(1, tmp.listFiles().length); // Ephemeral checkpoint - CheckpointProperties ephemeral = new CheckpointProperties(false, false, true, true, true, true, true); + CheckpointProperties ephemeral = new CheckpointProperties(false, false, false, true, true, true, true, true); pending = createPendingCheckpoint(ephemeral, null); pending.acknowledgeTask(ATTEMPT_ID, null, new CheckpointMetrics()); @@ -130,7 +130,7 @@ public void testPersistExternally() throws Exception { */ @Test public void testCompletionFuture() throws Exception { - CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false); + CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false, false); // Abort declined PendingCheckpoint pending = createPendingCheckpoint(props, "ignored"); @@ -192,7 +192,7 @@ public void testCompletionFuture() throws Exception { @Test @SuppressWarnings("unchecked") public void testAbortDiscardsState() throws Exception { - CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false); + CheckpointProperties props = new CheckpointProperties(false, true, false, false, false, false, false, false); QueueExecutor executor = new QueueExecutor(); OperatorState state = mock(OperatorState.class); @@ -330,7 +330,7 @@ public void testNonNullSubtaskStateLeadsToStatefulTask() throws Exception { @Test public void testSetCanceller() { - final CheckpointProperties props = new CheckpointProperties(false, false, true, true, true, true, true); + final CheckpointProperties props = new CheckpointProperties(false, false, false, true, true, true, true, true); PendingCheckpoint aborted = createPendingCheckpoint(props, null); aborted.abortDeclined(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java index 85b151635efe9..d43283d992b55 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/RestoredCheckpointStatsTest.java @@ -31,7 +31,7 @@ public class RestoredCheckpointStatsTest { public void testSimpleAccess() throws Exception { long checkpointId = Integer.MAX_VALUE + 1L; long triggerTimestamp = Integer.MAX_VALUE + 1L; - CheckpointProperties props = new CheckpointProperties(true, true, false, false, true, false, true); + CheckpointProperties props = new CheckpointProperties(true, true, false, false, false, true, false, true); long restoreTimestamp = Integer.MAX_VALUE + 1L; String externalPath = "external-path"; From 6c6d90084c9be27eb8c43f0f642c76e4dec9a4f6 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Wed, 16 Aug 2017 18:21:49 +0200 Subject: [PATCH 024/129] [FLINK-7462] [docs] Add very obvious warning about outdated docs This closes #4553. --- docs/_config.yml | 7 ++++--- docs/_layouts/base.html | 6 ++---- docs/_layouts/plain.html | 5 +++++ 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index 548278363a416..5a92bb99f8ac5 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -46,11 +46,12 @@ download_url: "http://flink.apache.org/downloads.html" # please use a protocol relative URL here baseurl: //ci.apache.org/projects/flink/flink-docs-release-1.4 -# Flag whether this is the latest stable version or not. If not, a warning -# will be printed pointing to the docs of the latest stable version. -is_latest: true +# Flag whether this is a stable version or not. Used for the quickstart page. is_stable: false +# Flag to indicate whether an outdated warning should be shown. +show_outdated_warning: false + previous_docs: 1.3: http://ci.apache.org/projects/flink/flink-docs-release-1.3 1.2: http://ci.apache.org/projects/flink/flink-docs-release-1.2 diff --git a/docs/_layouts/base.html b/docs/_layouts/base.html index d51451a3543f3..691670c47fdc0 100644 --- a/docs/_layouts/base.html +++ b/docs/_layouts/base.html @@ -54,12 +54,10 @@ - {% if site.is_stable %} - {% unless site.is_latest %} -

+ {% if site.show_outdated_warning %} +
This documentation is for an out-of-date version of Apache Flink. We recommend you use the latest stable version.
- {% endunless %} {% endif %} diff --git a/docs/_layouts/plain.html b/docs/_layouts/plain.html index 63a6681ed4afd..e991f78f82be3 100644 --- a/docs/_layouts/plain.html +++ b/docs/_layouts/plain.html @@ -53,5 +53,10 @@

{{ page.title }}{% if page.is_beta %} Beta{% endif %}

+{% if site.show_outdated_warning %} + +{% endif %} {{ content }} From d0a150609b46cabfe7f5f0d760c465dcee5588fb Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Thu, 17 Aug 2017 12:04:09 +0200 Subject: [PATCH 025/129] [FLINK-7056][tests][hotfix] make sure the client and a created InputStream are closed If not and the server has not yet sent all data packets, it may still occupy the read lock and block any writing operations (also see FLINK-7467). This closes #4558. --- .../flink/runtime/blob/BlobClientTest.java | 59 +++++++++---------- .../runtime/blob/BlobServerDeleteTest.java | 11 ++-- .../flink/runtime/blob/BlobServerGetTest.java | 7 ++- .../flink/runtime/blob/BlobServerPutTest.java | 25 +++----- 4 files changed, 47 insertions(+), 55 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java index cfec4c5f51494..d511e86cfeacf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java @@ -139,30 +139,35 @@ private static BlobKey prepareTestFile(File file) throws IOException { * the specified buffer. * * @param inputStream - * the input stream returned from the GET operation + * the input stream returned from the GET operation (will be closed by this method) * @param buf * the buffer to compare the input stream's data to * @throws IOException * thrown if an I/O error occurs while reading the input stream */ - static void validateGet(final InputStream inputStream, final byte[] buf) throws IOException { - byte[] receivedBuffer = new byte[buf.length]; + static void validateGetAndClose(final InputStream inputStream, final byte[] buf) throws IOException { + try { + byte[] receivedBuffer = new byte[buf.length]; - int bytesReceived = 0; + int bytesReceived = 0; - while (true) { + while (true) { - final int read = inputStream.read(receivedBuffer, bytesReceived, receivedBuffer.length - bytesReceived); - if (read < 0) { - throw new EOFException(); - } - bytesReceived += read; + final int read = inputStream + .read(receivedBuffer, bytesReceived, receivedBuffer.length - bytesReceived); + if (read < 0) { + throw new EOFException(); + } + bytesReceived += read; - if (bytesReceived == receivedBuffer.length) { - assertEquals(-1, inputStream.read()); - assertArrayEquals(buf, receivedBuffer); - return; + if (bytesReceived == receivedBuffer.length) { + assertEquals(-1, inputStream.read()); + assertArrayEquals(buf, receivedBuffer); + return; + } } + } finally { + inputStream.close(); } } @@ -171,13 +176,13 @@ static void validateGet(final InputStream inputStream, final byte[] buf) throws * the specified file. * * @param inputStream - * the input stream returned from the GET operation + * the input stream returned from the GET operation (will be closed by this method) * @param file * the file to compare the input stream's data to * @throws IOException * thrown if an I/O error occurs while reading the input stream or the file */ - private static void validateGet(final InputStream inputStream, final File file) throws IOException { + private static void validateGetAndClose(final InputStream inputStream, final File file) throws IOException { InputStream inputStream2 = null; try { @@ -200,6 +205,7 @@ private static void validateGet(final InputStream inputStream, final File file) if (inputStream2 != null) { inputStream2.close(); } + inputStream.close(); } } @@ -231,14 +237,11 @@ public void testContentAddressableBuffer() { assertEquals(origKey, receivedKey); // Retrieve the data - InputStream is = client.get(receivedKey); - validateGet(is, testBuffer); - is = client.get(jobId, receivedKey); - validateGet(is, testBuffer); + validateGetAndClose(client.get(receivedKey), testBuffer); + validateGetAndClose(client.get(jobId, receivedKey), testBuffer); // Check reaction to invalid keys - try { - client.get(new BlobKey()); + try (InputStream ignored = client.get(new BlobKey())) { fail("Expected IOException did not occur"); } catch (IOException fnfe) { @@ -246,8 +249,7 @@ public void testContentAddressableBuffer() { } // new client needed (closed from failure above) client = new BlobClient(serverAddress, getBlobClientConfig()); - try { - client.get(jobId, new BlobKey()); + try (InputStream ignored = client.get(jobId, new BlobKey())) { fail("Expected IOException did not occur"); } catch (IOException fnfe) { @@ -308,10 +310,8 @@ public void testContentAddressableStream() { is = null; // Retrieve the data - is = client.get(receivedKey); - validateGet(is, testFile); - is = client.get(jobId, receivedKey); - validateGet(is, testFile); + validateGetAndClose(client.get(receivedKey), testFile); + validateGetAndClose(client.get(jobId, receivedKey), testFile); } catch (Exception e) { e.printStackTrace(); @@ -362,8 +362,7 @@ private static void uploadJarFile( assertEquals(1, blobKeys.size()); try (BlobClient blobClient = new BlobClient(serverAddress, blobClientConfig)) { - InputStream is = blobClient.get(blobKeys.get(0)); - validateGet(is, testFile); + validateGetAndClose(blobClient.get(blobKeys.get(0)), testFile); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java index d91aae420f852..413e2e9f2ed78 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java @@ -31,6 +31,7 @@ import java.io.File; import java.io.IOException; +import java.io.InputStream; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; @@ -40,6 +41,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import static org.apache.flink.runtime.blob.BlobClientTest.validateGetAndClose; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; @@ -99,8 +101,7 @@ public void testDeleteSingleByBlobKey() { client.close(); client = new BlobClient(serverAddress, config); - try { - client.get(key1); + try (InputStream ignored = client.get(key1)) { fail("BLOB should have been deleted"); } catch (IOException e) { @@ -111,12 +112,14 @@ public void testDeleteSingleByBlobKey() { client = new BlobClient(serverAddress, config); try { - client.get(jobId, key1); + // NOTE: the server will stall in its send operation until either the data is fully + // read or the socket is closed, e.g. via a client.close() call + validateGetAndClose(client.get(jobId, key1), data); } catch (IOException e) { - // expected fail("Deleting a job-unrelated BLOB should not affect a job-related BLOB with the same key"); } + client.close(); // delete a file directly on the server server.delete(key2); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java index 5ad8d95928a1c..7ccf075937085 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerGetTest.java @@ -49,6 +49,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import static org.apache.flink.runtime.blob.BlobClientTest.validateGetAndClose; import static org.junit.Assert.*; import static org.mockito.Matchers.any; import static org.mockito.Matchers.eq; @@ -122,7 +123,7 @@ private void testGetFailsDuringLookup(final JobID jobId1, final JobID jobId2) assertNotNull(key); assertEquals(key, key2); // request for jobId2 should succeed - getFileHelper(client, jobId2, key); + validateGetAndClose(getFileHelper(client, jobId2, key), data); // request for jobId1 should still fail client = verifyDeleted(client, jobId1, key, serverAddress, config); @@ -160,8 +161,7 @@ private void testGetFailsDuringLookup(final JobID jobId1, final JobID jobId2) private static BlobClient verifyDeleted( BlobClient client, JobID jobId, BlobKey key, InetSocketAddress serverAddress, Configuration config) throws IOException { - try { - getFileHelper(client, jobId, key); + try (InputStream ignored = getFileHelper(client, jobId, key)) { fail("This should not succeed."); } catch (IOException e) { // expected @@ -227,6 +227,7 @@ private void testGetFailsDuringStreaming(final JobID jobId) throws IOException { catch (IOException e) { // expected } + is.close(); } finally { if (client != null) { client.close(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java index f55adb7a34c3c..2b8e2d27c0443 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerPutTest.java @@ -46,8 +46,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import static org.apache.flink.runtime.blob.BlobClientTest.validateGetAndClose; import static org.apache.flink.runtime.blob.BlobServerGetTest.getFileHelper; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; @@ -226,9 +226,9 @@ private void testPutBufferSuccessfulGet(final JobID jobId1, final JobID jobId2) * @param jobId * job ID or null if job-unrelated * @param key1 - * first key + * first key for 44 bytes starting at byte 10 of data in the BLOB * @param key2 - * second key + * second key for the complete data in the BLOB * @param data * expected data * @param serverAddress @@ -241,12 +241,9 @@ private static void testPutBufferSuccessfulGet( InetSocketAddress serverAddress, Configuration config) throws IOException { BlobClient client = new BlobClient(serverAddress, config); - InputStream is1 = null; - InputStream is2 = null; - try { - // one get request on the same client - is1 = getFileHelper(client, jobId, key2); + // one get request on the same client + try (InputStream is1 = getFileHelper(client, jobId, key2)) { byte[] result1 = new byte[44]; BlobUtils.readFully(is1, result1, 0, result1.length, null); is1.close(); @@ -255,20 +252,12 @@ private static void testPutBufferSuccessfulGet( assertEquals(data[j], result1[i]); } - // close the client and create a new one for the remaining requests + // close the client and create a new one for the remaining request client.close(); client = new BlobClient(serverAddress, config); - is2 = getFileHelper(client, jobId, key1); - BlobClientTest.validateGet(is2, data); - is2.close(); + validateGetAndClose(getFileHelper(client, jobId, key1), data); } finally { - if (is1 != null) { - is1.close(); - } - if (is2 != null) { - is1.close(); - } client.close(); } } From 9c80d407c6b31cf449572ad8cc90da0f15fb2b16 Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 4 Jul 2017 11:50:07 +0200 Subject: [PATCH 026/129] [hotfix] increase Scala checkstyle maxParameters to 20 --- tools/maven/scalastyle-config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/maven/scalastyle-config.xml b/tools/maven/scalastyle-config.xml index 0f7f6bbcb0484..848b2afc1ba0f 100644 --- a/tools/maven/scalastyle-config.xml +++ b/tools/maven/scalastyle-config.xml @@ -86,7 +86,7 @@ - + From 7b23624066c46d58c7b7181e5576a9834af9ac7a Mon Sep 17 00:00:00 2001 From: Nico Kruber Date: Tue, 27 Jun 2017 18:29:44 +0200 Subject: [PATCH 027/129] [FLINK-7057][blob] move ref-counting from the LibraryCacheManager to the BlobCache Also change from BlobKey-based ref-counting to job-based ref-counting which is simpler and the mode we want to use from now on. Deferred cleanup (as before) is currently not implemented yet (TODO). At the BlobServer, no ref-counting will be used but the cleanup will happen when the job enters a final state (TODO). [FLINK-7057][blob] change to a cleaner API for BlobService#registerJob() [FLINK-7057][blob] implement deferred cleanup at the BlobCache Whenever a job is not referenced at the BlobCache anymore, we set a TTL and let the cleanup task remove it when this is hit and the task is run. For now, this means that a BLOB will be retained at most (2 * ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) seconds after not being referenced anymore. We do this so that a recovery still has the chance to use existing files rather than to download them again. [FLINK-7057][blob] integrate cleanup of job-related JARs from the BlobServer TODO: an integration test that verifies that this is actually done when desired and not performed when not, e.g. if the job did not reach a final execution state [FLINK-7057][tests] extract FailingBlockingInvokable from CoordinatorShutdownTest [FLINK-7057][blob] add an integration test for the BlobServer cleanup This ensures that BLOB files are actually deleted when a job enters a final state. [FLINK-7057][tests] refrain from catching an exception just to fail the test removes code like this in the BLOB store unit tests: catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); } [FLINK-7057][blob] fix BlobServer#cleanupJob() being too eager Instead of deleting the job's directory, it was deleting the parent storage directory. [FLINK-7057][blob] fix BlobServer cleanup integration * the test did not check the correct directories for cleanup * the test did not honour the test timeout [FLINK-7057][blob] test and fix BlobServer cleanup for a failed job submission [FLINK-7057][blob] rework the LibraryCacheManager API Since ref-counting has moved to the BlobCache, the BlobLibraryCacheManager is just a thin wrapper to get a user class loader by retrieving BLOBs from the BlobCache/BlobServer. Therefore, move the job-registration/-release out of it, too, and restrict its use to the task manager where the BlobCache is used (on the BlobServer, jobs do not need registration since they are only used once and will be deleted when they enter a final state). This makes the BlobServer and BlobCache instances available at the JobManager and TaskManager instances, respectively, also enabling future use cases outside of the LibraryCacheManager. [FLINK-7057][blob] address PR comments [FLINK-7057][blob] fix JobManagerLeaderElectionTest [FLINK-7057][blob] re-introduce some ref-counting for BlobLibraryCacheManager Apparently, we do need to return the same ClassLoader for different (parallel) tasks of a job running on the same task manager. Therefore, keep the initial task registration implementation that was removed with 8331fbb208d975e0c1ec990344c14315ea08dd4a and only adapt it here. This also restores some tests and adds new combinations not tested before. [FLINK-7057][blob] address PR comments [FLINK-7057][tests] fix (manual/ignored) BlobCacheCleanupTest#testJobDeferredCleanup() [FLINK-7057][hotfix] fix a checkstyle error [FLINK-7057][blob] remove the extra lock object from BlobCache We can lock on jobRefCounters instead, which is what we are guarding anyway. [FLINK-7057][blob] minor improvements to the TTL in BlobCache Do not use Long.MAX_VALUE as a code for "keep forever". Also add more comments. [FLINK-7057][blob] replace "library-cache-manager.cleanup.interval" with "blob.service.cleanup.interval" Since we moved the cleanup to the BLOB service classes, this only makes sense. [FLINK-7057][hotfix] remove an unused import [FLINK-7057][docs] adapt javadocs of JobManager descendents [FLINK-7057][blob] increase JobManagerCleanupITCase timeout The previous value of 15s seems to be too low for some runs on Travis. [FLINK-7057][blob] providing more debug output in JobManagerCleanupITCase In case the BlobServer's directory is not cleaned within the remaining time, also print which files remain. This may help debugging the situation. This closes #4238. --- docs/ops/config.md | 7 + .../configuration/BlobServerOptions.java | 16 +- .../flink/configuration/ConfigConstants.java | 9 +- .../clusterframework/MesosJobManager.scala | 8 +- .../apache/flink/runtime/blob/BlobCache.java | 140 ++++- .../apache/flink/runtime/blob/BlobClient.java | 21 +- .../apache/flink/runtime/blob/BlobServer.java | 40 +- .../runtime/blob/BlobServerConnection.java | 18 +- .../flink/runtime/blob/BlobService.java | 5 +- .../apache/flink/runtime/blob/BlobUtils.java | 25 +- .../flink/runtime/client/JobClient.java | 3 +- .../flink/runtime/dispatcher/Dispatcher.java | 3 +- .../dispatcher/StandaloneDispatcher.java | 5 +- .../entrypoint/JobClusterEntrypoint.java | 3 +- .../librarycache/BlobLibraryCacheManager.java | 327 +++++------ .../FallbackLibraryCacheManager.java | 8 +- .../librarycache/LibraryCacheManager.java | 25 +- .../flink/runtime/jobgraph/JobGraph.java | 2 +- .../runtime/jobmaster/JobManagerRunner.java | 7 +- .../runtime/jobmaster/JobManagerServices.java | 20 +- .../flink/runtime/jobmaster/JobMaster.java | 14 +- .../taskexecutor/JobManagerConnection.java | 32 +- .../runtime/taskexecutor/TaskExecutor.java | 11 +- .../TaskManagerConfiguration.java | 12 +- .../flink/runtime/taskmanager/Task.java | 13 +- .../ContaineredJobManager.scala | 6 +- .../flink/runtime/jobmanager/JobManager.scala | 48 +- .../minicluster/LocalFlinkMiniCluster.scala | 5 + .../runtime/taskmanager/TaskManager.scala | 34 +- .../runtime/blob/BlobCacheCleanupTest.java | 328 +++++++++++ .../runtime/blob/BlobCacheRetriesTest.java | 4 +- .../flink/runtime/blob/BlobClientTest.java | 29 +- .../flink/runtime/blob/BlobKeyTest.java | 6 +- .../runtime/blob/BlobServerDeleteTest.java | 85 ++- .../flink/runtime/blob/BlobUtilsTest.java | 3 +- .../checkpoint/CoordinatorShutdownTest.java | 23 +- .../runtime/dispatcher/DispatcherTest.java | 3 +- .../BlobLibraryCacheManagerTest.java | 540 ++++++++++++------ .../BlobLibraryCacheRecoveryITCase.java | 36 +- .../jobmanager/JobManagerCleanupITCase.java | 300 ++++++++++ .../jobmanager/JobManagerHARecoveryTest.java | 20 +- .../runtime/jobmanager/JobSubmitTest.java | 5 +- .../runtime/jobmaster/JobMasterTest.java | 9 +- .../JobManagerLeaderElectionTest.java | 11 +- .../taskexecutor/TaskExecutorTest.java | 5 +- .../taskmanager/TaskAsyncCallTest.java | 3 + .../runtime/taskmanager/TaskStopTest.java | 2 + .../flink/runtime/taskmanager/TaskTest.java | 35 +- .../testtasks/FailingBlockingInvokable.java | 48 ++ .../runtime/util/JvmExitOnFatalErrorTest.java | 2 + .../JobManagerRegistrationTest.scala | 5 +- .../runtime/testingUtils/TestingCluster.scala | 3 + .../testingUtils/TestingJobManager.scala | 9 +- .../tasks/BlockingCheckpointsTest.java | 2 + .../tasks/InterruptSensitiveRestoreTest.java | 2 + .../tasks/StreamTaskTerminationTest.java | 2 + .../runtime/tasks/StreamTaskTest.java | 3 + .../flink/yarn/TestingYarnJobManager.scala | 3 + .../apache/flink/yarn/YarnJobManager.scala | 6 +- 59 files changed, 1759 insertions(+), 640 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerCleanupITCase.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/FailingBlockingInvokable.java diff --git a/docs/ops/config.md b/docs/ops/config.md index 4138b4d270ef3..e0b9d4db714f5 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -196,6 +196,13 @@ will be used under the directory specified by jobmanager.web.tmpdir. - `blob.storage.directory`: Directory for storing blobs (such as user JARs) on the TaskManagers. +- `blob.service.cleanup.interval`: Cleanup interval (in seconds) of the blob caches (DEFAULT: 1 hour). +Whenever a job is not referenced at the cache anymore, we set a TTL and let the periodic cleanup task +(executed every `blob.service.cleanup.interval` seconds) remove its blob files after this TTL has passed. +This means that a blob will be retained at most 2 * `blob.service.cleanup.interval` seconds after +not being referenced anymore. Therefore, a recovery still has the chance to use existing files rather +than to download them again. + - `blob.server.port`: Port definition for the blob server (serving user JARs) on the TaskManagers. By default the port is set to 0, which means that the operating system is picking an ephemeral port. Flink also accepts a list of ports ("50100,50101"), ranges ("50100-50200") or a combination of both. It is recommended to set a range of ports to avoid collisions when multiple JobManagers are running on the same machine. - `blob.service.ssl.enabled`: Flag to enable ssl for the blob client/server communication. This is applicable only when the global ssl flag security.ssl.enabled is set to true (DEFAULT: true). diff --git a/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java index e27c29f2e0b46..019580a6705dd 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/BlobServerOptions.java @@ -22,7 +22,7 @@ import static org.apache.flink.configuration.ConfigOptions.key; /** - * Configuration options for the BlobServer. + * Configuration options for the BlobServer and BlobCache. */ @PublicEvolving public class BlobServerOptions { @@ -73,4 +73,18 @@ public class BlobServerOptions { public static final ConfigOption SSL_ENABLED = key("blob.service.ssl.enabled") .defaultValue(true); + + /** + * Cleanup interval of the blob caches at the task managers (in seconds). + * + *

Whenever a job is not referenced at the cache anymore, we set a TTL and let the periodic + * cleanup task (executed every CLEANUP_INTERVAL seconds) remove its blob files after this TTL + * has passed. This means that a blob will be retained at most 2 * CLEANUP_INTERVAL + * seconds after not being referenced anymore. Therefore, a recovery still has the chance to use + * existing files rather than to download them again. + */ + public static final ConfigOption CLEANUP_INTERVAL = + key("blob.service.cleanup.interval") + .defaultValue(3_600L) // once per hour + .withDeprecatedKeys("library-cache-manager.cleanup.interval"); } diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index 4c6c62a51c8ca..4153e456e3160 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -178,7 +178,10 @@ public final class ConfigConstants { /** * The config parameter defining the cleanup interval of the library cache manager. + * + * @deprecated use {@link BlobServerOptions#CLEANUP_INTERVAL} instead */ + @Deprecated public static final String LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL = "library-cache-manager.cleanup.interval"; /** @@ -1253,8 +1256,12 @@ public final class ConfigConstants { /** * The default library cache manager cleanup interval in seconds + * + * @deprecated use {@link BlobServerOptions#CLEANUP_INTERVAL} instead */ - public static final long DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL = 3600; + @Deprecated + public static final long DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL = + BlobServerOptions.CLEANUP_INTERVAL.defaultValue(); /** * The default network port to connect to for communication with the job manager. diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala index 3e7c55f0bf957..f854a1e320396 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{Executor, ScheduledExecutorService} import akka.actor.ActorRef import org.apache.flink.configuration.{Configuration => FlinkConfiguration} +import org.apache.flink.runtime.blob.BlobServer import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.clusterframework.ContaineredJobManager import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager @@ -34,7 +35,7 @@ import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry} import scala.concurrent.duration._ -/** JobManager actor for execution on Mesos. . +/** JobManager actor for execution on Mesos. * * @param flinkConfiguration Configuration object for the actor * @param futureExecutor Execution context which is used to execute concurrent tasks in the @@ -43,7 +44,8 @@ import scala.concurrent.duration._ * @param instanceManager Instance manager to manage the registered * [[org.apache.flink.runtime.taskmanager.TaskManager]] * @param scheduler Scheduler to schedule Flink jobs - * @param libraryCacheManager Manager to manage uploaded jar files + * @param blobServer BLOB store for file uploads + * @param libraryCacheManager manages uploaded jar files and class paths * @param archive Archive for finished Flink jobs * @param restartStrategyFactory Restart strategy to be used in case of a job recovery * @param timeout Timeout for futures @@ -55,6 +57,7 @@ class MesosJobManager( ioExecutor: Executor, instanceManager: InstanceManager, scheduler: FlinkScheduler, + blobServer: BlobServer, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, restartStrategyFactory: RestartStrategyFactory, @@ -70,6 +73,7 @@ class MesosJobManager( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java index 29f7706154de2..c50a8887f8558 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobCache.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; @@ -25,7 +26,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.File; import java.io.FileOutputStream; @@ -33,6 +33,11 @@ import java.io.InputStream; import java.io.OutputStream; import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Timer; +import java.util.TimerTask; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.util.Preconditions.checkArgument; @@ -47,7 +52,7 @@ * download it from a distributed file system (if available) or the BLOB * server.

*/ -public final class BlobCache implements BlobService { +public class BlobCache extends TimerTask implements BlobService { /** The log object used for debugging. */ private static final Logger LOG = LoggerFactory.getLogger(BlobCache.class); @@ -71,6 +76,32 @@ public final class BlobCache implements BlobService { /** Configuration for the blob client like ssl parameters required to connect to the blob server */ private final Configuration blobClientConfig; + // -------------------------------------------------------------------------------------------- + + /** + * Job reference counters with a time-to-live (TTL). + */ + private static class RefCount { + /** + * Number of references to a job. + */ + public int references = 0; + + /** + * Timestamp in milliseconds when any job data should be cleaned up (no cleanup for + * non-positive values). + */ + public long keepUntil = -1; + } + + /** Map to store the number of references to a specific job */ + private final Map jobRefCounters = new HashMap<>(); + + /** Time interval (ms) to run the cleanup task; also used as the default TTL. */ + private final long cleanupInterval; + + private final Timer cleanupTimer; + /** * Instantiates a new BLOB cache. * @@ -108,10 +139,62 @@ public BlobCache( this.numFetchRetries = 0; } + // Initializing the clean up task + this.cleanupTimer = new Timer(true); + + cleanupInterval = blobClientConfig.getLong(BlobServerOptions.CLEANUP_INTERVAL) * 1000; + this.cleanupTimer.schedule(this, cleanupInterval, cleanupInterval); + // Add shutdown hook to delete storage directory shutdownHook = BlobUtils.addShutdownHook(this, LOG); } + /** + * Registers use of job-related BLOBs. + *

+ * Using any other method to access BLOBs, e.g. {@link #getFile}, is only valid within calls + * to {@link #registerJob(JobID)} and {@link #releaseJob(JobID)}. + * + * @param jobId + * ID of the job this blob belongs to + * + * @see #releaseJob(JobID) + */ + public void registerJob(JobID jobId) { + synchronized (jobRefCounters) { + RefCount ref = jobRefCounters.get(jobId); + if (ref == null) { + ref = new RefCount(); + jobRefCounters.put(jobId, ref); + } + ++ref.references; + } + } + + /** + * Unregisters use of job-related BLOBs and allow them to be released. + * + * @param jobId + * ID of the job this blob belongs to + * + * @see #registerJob(JobID) + */ + public void releaseJob(JobID jobId) { + synchronized (jobRefCounters) { + RefCount ref = jobRefCounters.get(jobId); + + if (ref == null) { + LOG.warn("improper use of releaseJob() without a matching number of registerJob() calls"); + return; + } + + --ref.references; + if (ref.references == 0) { + ref.keepUntil = System.currentTimeMillis() + cleanupInterval; + } + } + } + /** * Returns local copy of the (job-unrelated) file for the BLOB with the given key. *

@@ -148,7 +231,7 @@ public File getFile(BlobKey key) throws IOException { * Thrown if an I/O error occurs while downloading the BLOBs from the BLOB server. */ @Override - public File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException { + public File getFile(JobID jobId, BlobKey key) throws IOException { checkNotNull(jobId); return getFileInternal(jobId, key); } @@ -258,7 +341,7 @@ public void delete(BlobKey key) throws IOException { * @throws IOException */ @Override - public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException { + public void delete(JobID jobId, BlobKey key) throws IOException { checkNotNull(jobId); deleteInternal(jobId, key); } @@ -307,7 +390,7 @@ public void deleteGlobal(BlobKey key) throws IOException { * thrown if an I/O error occurs while transferring the request to the BLOB server or if the * BLOB server cannot delete the file */ - public void deleteGlobal(@Nonnull JobID jobId, BlobKey key) throws IOException { + public void deleteGlobal(JobID jobId, BlobKey key) throws IOException { checkNotNull(jobId); deleteGlobalInternal(jobId, key); } @@ -341,8 +424,40 @@ public int getPort() { return serverAddress.getPort(); } + /** + * Cleans up BLOBs which are not referenced anymore. + */ + @Override + public void run() { + synchronized (jobRefCounters) { + Iterator> entryIter = jobRefCounters.entrySet().iterator(); + final long currentTimeMillis = System.currentTimeMillis(); + + while (entryIter.hasNext()) { + Map.Entry entry = entryIter.next(); + RefCount ref = entry.getValue(); + + if (ref.references <= 0 && ref.keepUntil > 0 && currentTimeMillis >= ref.keepUntil) { + JobID jobId = entry.getKey(); + + final File localFile = + new File(BlobUtils.getStorageLocationPath(storageDir.getAbsolutePath(), jobId)); + try { + FileUtils.deleteDirectory(localFile); + // let's only remove this directory from cleanup if the cleanup was successful + entryIter.remove(); + } catch (Throwable t) { + LOG.warn("Failed to locally delete job directory " + localFile.getAbsolutePath(), t); + } + } + } + } + } + @Override public void close() throws IOException { + cleanupTimer.cancel(); + if (shutdownRequested.compareAndSet(false, true)) { LOG.info("Shutting down BlobCache"); @@ -369,8 +484,19 @@ public BlobClient createClient() throws IOException { return new BlobClient(serverAddress, blobClientConfig); } - public File getStorageDir() { - return this.storageDir; + /** + * Returns a file handle to the file associated with the given blob key on the blob + * server. + * + *

This is only called from the {@link BlobServerConnection} + * + * @param jobId ID of the job this blob belongs to (or null if job-unrelated) + * @param key identifying the file + * @return file handle to the file + */ + @VisibleForTesting + public File getStorageLocation(JobID jobId, BlobKey key) { + return BlobUtils.getStorageLocation(storageDir, jobId, key); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java index 9a2f59e31cf14..8f1487ae28c5b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java @@ -30,7 +30,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.net.ssl.SSLContext; import javax.net.ssl.SSLParameters; @@ -166,7 +165,7 @@ public InputStream get(BlobKey blobKey) throws IOException { * @throws IOException * if an I/O error occurs during the download */ - public InputStream get(@Nonnull JobID jobId, BlobKey blobKey) throws IOException { + public InputStream get(JobID jobId, BlobKey blobKey) throws IOException { checkNotNull(jobId); return getInternal(jobId, blobKey); } @@ -339,7 +338,7 @@ public BlobKey put(InputStream inputStream) throws IOException { * thrown if an I/O error occurs while reading the data from the input stream or uploading the * data to the BLOB server */ - public BlobKey put(@Nonnull JobID jobId, InputStream inputStream) throws IOException { + public BlobKey put(JobID jobId, InputStream inputStream) throws IOException { checkNotNull(jobId); return putInputStream(jobId, inputStream); } @@ -369,7 +368,7 @@ private BlobKey putBuffer(@Nullable JobID jobId, byte[] value, int offset, int l checkNotNull(value); if (LOG.isDebugEnabled()) { - LOG.debug("PUT BLOB buffer ({} bytes) to {}.", len, socket.getLocalSocketAddress()); + LOG.debug("PUT BLOB buffer (" + len + " bytes) to " + socket.getLocalSocketAddress() + "."); } try { @@ -556,7 +555,7 @@ public void delete(BlobKey key) throws IOException { * thrown if an I/O error occurs while transferring the request to the BLOB server or if the * BLOB server cannot delete the file */ - public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException { + public void delete(JobID jobId, BlobKey key) throws IOException { checkNotNull(jobId); deleteInternal(jobId, key); } @@ -603,23 +602,21 @@ public void deleteInternal(@Nullable JobID jobId, BlobKey key) throws IOExceptio /** * Uploads the JAR files to a {@link BlobServer} at the given address. - *

- * TODO: add jobId to signature after adapting the BlobLibraryCacheManager * * @param serverAddress * Server address of the {@link BlobServer} * @param clientConfig * Any additional configuration for the blob client + * @param jobId + * ID of the job this blob belongs to (or null if job-unrelated) * @param jars * List of JAR files to upload * * @throws IOException * if the upload fails */ - public static List uploadJarFiles( - InetSocketAddress serverAddress, - Configuration clientConfig, - List jars) throws IOException { + public static List uploadJarFiles(InetSocketAddress serverAddress, + Configuration clientConfig, JobID jobId, List jars) throws IOException {checkNotNull(jobId); if (jars.isEmpty()) { return Collections.emptyList(); } else { @@ -631,7 +628,7 @@ public static List uploadJarFiles( FSDataInputStream is = null; try { is = fs.open(jar); - final BlobKey key = blobClient.putInputStream(null, is); + final BlobKey key = blobClient.putInputStream(jobId, is); blobKeys.add(key); } finally { if (is != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java index 43a060a96f97a..bfcf881dbea75 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.blob; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; @@ -29,7 +30,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import javax.net.ssl.SSLContext; import java.io.File; @@ -196,7 +196,8 @@ public ServerSocket createSocket(int port) throws IOException { * @param key identifying the file * @return file handle to the file */ - File getStorageLocation(JobID jobId, BlobKey key) { + @VisibleForTesting + public File getStorageLocation(JobID jobId, BlobKey key) { return BlobUtils.getStorageLocation(storageDir, jobId, key); } @@ -374,7 +375,7 @@ public File getFile(BlobKey key) throws IOException { * Thrown if the file retrieval failed. */ @Override - public File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException { + public File getFile(JobID jobId, BlobKey key) throws IOException { checkNotNull(jobId); return getFileInternal(jobId, key); } @@ -450,7 +451,7 @@ public void delete(BlobKey key) throws IOException { * @throws IOException */ @Override - public void delete(@Nonnull JobID jobId, BlobKey key) throws IOException { + public void delete(JobID jobId, BlobKey key) throws IOException { checkNotNull(jobId); deleteInternal(jobId, key); } @@ -482,6 +483,37 @@ void deleteInternal(@Nullable JobID jobId, BlobKey key) throws IOException { } } + /** + * Removes all BLOBs from local and HA store belonging to the given job ID. + * + * @param jobId + * ID of the job this blob belongs to + */ + public void cleanupJob(JobID jobId) { + checkNotNull(jobId); + + final File jobDir = + new File(BlobUtils.getStorageLocationPath(storageDir.getAbsolutePath(), jobId)); + + readWriteLock.writeLock().lock(); + + try { + // delete locally + try { + FileUtils.deleteDirectory(jobDir); + } catch (IOException e) { + LOG.warn("Failed to locally delete BLOB storage directory at " + + jobDir.getAbsolutePath(), e); + } + + // delete in HA store + blobStore.deleteAll(jobId); + } finally { + readWriteLock.writeLock().unlock(); + } + } + + /** * Returns the port on which the server is listening. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java index f1054c090f1c3..7f617f91d0ca1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServerConnection.java @@ -139,14 +139,7 @@ public void run() { LOG.error("Error while executing BLOB connection.", t); } finally { - try { - if (clientSocket != null) { - clientSocket.close(); - } - } catch (Throwable t) { - LOG.debug("Exception while closing BLOB server connection socket.", t); - } - + closeSilently(clientSocket, LOG); blobServer.unregisterConnection(this); } } @@ -433,9 +426,8 @@ private static BlobKey readFileFully( final InputStream inputStream, final File incomingFile, final byte[] buf) throws IOException { MessageDigest md = BlobUtils.createMessageDigest(); - FileOutputStream fos = new FileOutputStream(incomingFile); - try { + try (FileOutputStream fos = new FileOutputStream(incomingFile)) { while (true) { final int bytesExpected = readLength(inputStream); if (bytesExpected == -1) { @@ -453,12 +445,6 @@ private static BlobKey readFileFully( md.update(buf, 0, bytesExpected); } return new BlobKey(md.digest()); - } finally { - try { - fos.close(); - } catch (Throwable t) { - LOG.warn("Cannot close stream to BLOB staging file", t); - } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java index a78c88c8e89b3..0db5a589593b8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobService.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; -import javax.annotation.Nonnull; import java.io.Closeable; import java.io.File; import java.io.IOException; @@ -50,7 +49,7 @@ public interface BlobService extends Closeable { * @throws java.io.FileNotFoundException when the path does not exist; * @throws IOException if any other error occurs when retrieving the file */ - File getFile(@Nonnull JobID jobId, BlobKey key) throws IOException; + File getFile(JobID jobId, BlobKey key) throws IOException; /** * Deletes the (job-unrelated) file associated with the provided blob key. @@ -67,7 +66,7 @@ public interface BlobService extends Closeable { * @param key associated with the file to be deleted * @throws IOException */ - void delete(@Nonnull JobID jobId, BlobKey key) throws IOException; + void delete(JobID jobId, BlobKey key) throws IOException; /** * Returns the port of the blob service. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java index 9b5724b084518..dabd1bfd764cb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java @@ -29,8 +29,8 @@ import org.slf4j.Logger; -import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.EOFException; import java.io.File; import java.io.IOException; @@ -175,7 +175,7 @@ static File initLocalStorageDirectory(String basePath) throws IOException { static File getIncomingDirectory(File storageDir) { final File incomingDir = new File(storageDir, "incoming"); - mkdirTolerateExisting(incomingDir, "incoming"); + mkdirTolerateExisting(incomingDir); return incomingDir; } @@ -185,15 +185,13 @@ static File getIncomingDirectory(File storageDir) { * * @param dir * directory to create - * @param dirType - * the type of the directory (included in error message if something fails) */ - private static void mkdirTolerateExisting(final File dir, final String dirType) { + private static void mkdirTolerateExisting(final File dir) { // note: thread-safe create should try to mkdir first and then ignore the case that the // directory already existed if (!dir.mkdirs() && !dir.exists()) { throw new RuntimeException( - "Cannot create " + dirType + " directory '" + dir.getAbsolutePath() + "'."); + "Cannot create directory '" + dir.getAbsolutePath() + "'."); } } @@ -210,10 +208,10 @@ private static void mkdirTolerateExisting(final File dir, final String dirType) * @return the (designated) physical storage location of the BLOB */ static File getStorageLocation( - @Nonnull File storageDir, @Nullable JobID jobId, @Nonnull BlobKey key) { + File storageDir, @Nullable JobID jobId, BlobKey key) { File file = new File(getStorageLocationPath(storageDir.getAbsolutePath(), jobId, key)); - mkdirTolerateExisting(file.getParentFile(), "cache"); + mkdirTolerateExisting(file.getParentFile()); return file; } @@ -229,7 +227,7 @@ static File getStorageLocation( * * @return the storage directory for BLOBs belonging to the job with the given ID */ - static String getStorageLocationPath(@Nonnull String storageDir, @Nullable JobID jobId) { + static String getStorageLocationPath(String storageDir, @Nullable JobID jobId) { if (jobId == null) { // format: $base/no_job return String.format("%s/%s", storageDir, NO_JOB_DIR_PREFIX); @@ -256,7 +254,7 @@ static String getStorageLocationPath(@Nonnull String storageDir, @Nullable JobID * @return the path to the given BLOB */ static String getStorageLocationPath( - @Nonnull String storageDir, @Nullable JobID jobId, @Nonnull BlobKey key) { + String storageDir, @Nullable JobID jobId, BlobKey key) { if (jobId == null) { // format: $base/no_job/blob_$key return String.format("%s/%s/%s%s", @@ -273,7 +271,6 @@ static String getStorageLocationPath( * * @return a new instance of the message digest to use for the BLOB key computation */ - @Nonnull static MessageDigest createMessageDigest() { try { return MessageDigest.getInstance(HASHING_ALGORITHM); @@ -285,7 +282,7 @@ static MessageDigest createMessageDigest() { /** * Adds a shutdown hook to the JVM and returns the Thread, which has been registered. */ - static Thread addShutdownHook(final BlobService service, final Logger logger) { + static Thread addShutdownHook(final Closeable service, final Logger logger) { checkNotNull(service); checkNotNull(logger); @@ -399,9 +396,7 @@ static void closeSilently(Socket socket, Logger LOG) { try { socket.close(); } catch (Throwable t) { - if (LOG.isDebugEnabled()) { - LOG.debug("Error while closing resource after BLOB transfer.", t); - } + LOG.debug("Exception while closing BLOB server connection socket.", t); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java index 9cc621094570a..425461cd58bb7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java @@ -234,8 +234,7 @@ public static ClassLoader retrieveClassLoader( int pos = 0; for (BlobKey blobKey : props.requiredJarFiles()) { try { - // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager - allURLs[pos++] = blobClient.getFile(blobKey).toURI().toURL(); + allURLs[pos++] = blobClient.getFile(jobID, blobKey).toURI().toURL(); } catch (Exception e) { try { blobClient.close(); 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 9fc1fc4098179..bb0b3e4ab3cf4 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 @@ -23,7 +23,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -231,7 +230,7 @@ protected abstract JobManagerRunner createJobManagerRunner( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - BlobService blobService, + BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, OnCompletionActions onCompleteActions, 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 54d698ef6c2d9..dfd6a8aac9bf1 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 @@ -20,7 +20,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -65,7 +64,7 @@ protected JobManagerRunner createJobManagerRunner( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - BlobService blobService, + BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, OnCompletionActions onCompleteActions, @@ -77,7 +76,7 @@ protected JobManagerRunner createJobManagerRunner( configuration, rpcService, highAvailabilityServices, - blobService, + blobServer, heartbeatServices, metricRegistry, onCompleteActions, 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 87281865e9684..a7c6120dced53 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 @@ -22,7 +22,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -90,7 +89,7 @@ protected JobManagerRunner createJobManagerRunner( ResourceID resourceId, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - BlobService blobService, + BlobServer blobService, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java index 9aff6f94c7b67..c8fc4e4c3a53b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManager.java @@ -23,9 +23,12 @@ import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.util.ExceptionUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.net.URL; import java.util.Arrays; @@ -33,72 +36,52 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.Map; import java.util.Set; -import java.util.Timer; -import java.util.TimerTask; +import java.util.stream.Collectors; import static org.apache.flink.util.Preconditions.checkNotNull; /** - * For each job graph that is submitted to the system the library cache manager maintains - * a set of libraries (typically JAR files) which the job requires to run. The library cache manager - * caches library files in order to avoid unnecessary retransmission of data. It is based on a singleton - * programming pattern, so there exists at most one library manager at a time. - *

- * All files registered via {@link #registerJob(JobID, Collection, Collection)} are reference-counted - * and are removed by a timer-based cleanup task if their reference counter is zero. + * Provides facilities to download a set of libraries (typically JAR files) for a job from a + * {@link BlobService} and create a class loader with references to them. */ -public final class BlobLibraryCacheManager extends TimerTask implements LibraryCacheManager { +public class BlobLibraryCacheManager implements LibraryCacheManager { + + private static final Logger LOG = LoggerFactory.getLogger(BlobLibraryCacheManager.class); + + private static final ExecutionAttemptID JOB_ATTEMPT_ID = new ExecutionAttemptID(-1, -1); - private static Logger LOG = LoggerFactory.getLogger(BlobLibraryCacheManager.class); - - private static ExecutionAttemptID JOB_ATTEMPT_ID = new ExecutionAttemptID(-1, -1); - // -------------------------------------------------------------------------------------------- - + /** The global lock to synchronize operations */ private final Object lockObject = new Object(); /** Registered entries per job */ - private final Map cacheEntries = new HashMap(); - - /** Map to store the number of reference to a specific file */ - private final Map blobKeyReferenceCounters = new HashMap(); + private final Map cacheEntries = new HashMap<>(); /** The blob service to download libraries */ private final BlobService blobService; - - private final Timer cleanupTimer; - + // -------------------------------------------------------------------------------------------- - /** - * Creates the blob library cache manager. - * - * @param blobService blob file retrieval service to use - * @param cleanupInterval cleanup interval in milliseconds - */ - public BlobLibraryCacheManager(BlobService blobService, long cleanupInterval) { + public BlobLibraryCacheManager(BlobService blobService) { this.blobService = checkNotNull(blobService); - - // Initializing the clean up task - this.cleanupTimer = new Timer(true); - this.cleanupTimer.schedule(this, cleanupInterval, cleanupInterval); } - // -------------------------------------------------------------------------------------------- - @Override public void registerJob(JobID id, Collection requiredJarFiles, Collection requiredClasspaths) - throws IOException { + throws IOException { registerTask(id, JOB_ATTEMPT_ID, requiredJarFiles, requiredClasspaths); } - + @Override - public void registerTask(JobID jobId, ExecutionAttemptID task, Collection requiredJarFiles, - Collection requiredClasspaths) throws IOException { + public void registerTask( + JobID jobId, + ExecutionAttemptID task, + @Nullable Collection requiredJarFiles, + @Nullable Collection requiredClasspaths) throws IOException { + checkNotNull(jobId, "The JobId must not be null."); checkNotNull(task, "The task execution id must not be null."); @@ -113,43 +96,31 @@ public void registerTask(JobID jobId, ExecutionAttemptID task, Collection> entryIter = blobKeyReferenceCounters.entrySet().iterator(); - - while (entryIter.hasNext()) { - Map.Entry entry = entryIter.next(); - BlobKey key = entry.getKey(); - int references = entry.getValue(); - - try { - if (references <= 0) { - blobService.delete(key); - entryIter.remove(); - } - } catch (Throwable t) { - LOG.warn("Could not delete file with blob key" + key, t); - } - } - } - } - - public int getNumberOfReferenceHolders(JobID jobId) { + int getNumberOfReferenceHolders(JobID jobId) { synchronized (lockObject) { LibraryCacheEntry entry = cacheEntries.get(jobId); return entry == null ? 0 : entry.getNumberOfReferenceHolders(); } } - - int getNumberOfCachedLibraries() { - return blobKeyReferenceCounters.size(); - } - - private URL registerReferenceToBlobKeyAndGetURL(BlobKey key) throws IOException { - // it is important that we fetch the URL before increasing the counter. - // in case the URL cannot be created (failed to fetch the BLOB), we have no stale counter - try { - URL url = blobService.getFile(key).toURI().toURL(); - Integer references = blobKeyReferenceCounters.get(key); - int newReferences = references == null ? 1 : references + 1; - blobKeyReferenceCounters.put(key, newReferences); - - return url; - } - catch (IOException e) { - throw new IOException("Cannot get library with hash " + key, e); - } + /** + * Returns the number of registered jobs that this library cache manager handles. + * + * @return number of jobs (irrespective of the actual number of tasks per job) + */ + int getNumberOfManagedJobs() { + // no synchronisation necessary + return cacheEntries.size(); } - - private void unregisterReferenceToBlobKey(BlobKey key) { - Integer references = blobKeyReferenceCounters.get(key); - if (references != null) { - int newReferences = Math.max(references - 1, 0); - blobKeyReferenceCounters.put(key, newReferences); - } - else { - // make sure we have an entry in any case, that the cleanup timer removes any - // present libraries - blobKeyReferenceCounters.put(key, 0); + + @Override + public void shutdown() { + synchronized (lockObject) { + for (LibraryCacheEntry entry : cacheEntries.values()) { + entry.releaseClassLoader(); + } } } - // -------------------------------------------------------------------------------------------- /** * An entry in the per-job library cache. Tracks which execution attempts * still reference the libraries. Once none reference it any more, the - * libraries can be cleaned up. + * class loaders can be cleaned up. */ private static class LibraryCacheEntry { - + private final FlinkUserCodeClassLoader classLoader; - + private final Set referenceHolders; - + /** + * Set of BLOB keys used for a previous job/task registration. + * + *

The purpose of this is to make sure, future registrations do not differ in content as + * this is a contract of the {@link BlobLibraryCacheManager}. + */ private final Set libraries; - - - public LibraryCacheEntry(Collection libraries, URL[] libraryURLs, ExecutionAttemptID initialReference) { + + /** + * Set of class path URLs used for a previous job/task registration. + * + *

The purpose of this is to make sure, future registrations do not differ in content as + * this is a contract of the {@link BlobLibraryCacheManager}. + */ + private final Set classPaths; + + /** + * Creates a cache entry for a flink class loader with the given libraryURLs. + * + * @param requiredLibraries + * BLOB keys required by the class loader (stored for ensuring consistency among different + * job/task registrations) + * @param requiredClasspaths + * class paths required by the class loader (stored for ensuring consistency among + * different job/task registrations) + * @param libraryURLs + * complete list of URLs to use for the class loader (includes references to the + * requiredLibraries and requiredClasspaths) + * @param initialReference + * reference holder ID + */ + LibraryCacheEntry( + Collection requiredLibraries, + Collection requiredClasspaths, + URL[] libraryURLs, + ExecutionAttemptID initialReference) { + this.classLoader = new FlinkUserCodeClassLoader(libraryURLs); - this.libraries = new HashSet<>(libraries); + // NOTE: do not store the class paths, i.e. URLs, into a set for performance reasons + // see http://findbugs.sourceforge.net/bugDescriptions.html#DMI_COLLECTION_OF_URLS + // -> alternatively, compare their string representation + this.classPaths = new HashSet<>(requiredClasspaths.size()); + for (URL url : requiredClasspaths) { + classPaths.add(url.toString()); + } + this.libraries = new HashSet<>(requiredLibraries); this.referenceHolders = new HashSet<>(); this.referenceHolders.add(initialReference); } - - + public ClassLoader getClassLoader() { return classLoader; } - + public Set getLibraries() { return libraries; } - - public void register(ExecutionAttemptID task, Collection keys) { - if (!libraries.containsAll(keys)) { + + public void register( + ExecutionAttemptID task, Collection requiredLibraries, + Collection requiredClasspaths) { + + // Make sure the previous registration referred to the same libraries and class paths. + // NOTE: the original collections may contain duplicates and may not already be Set + // collections with fast checks whether an item is contained in it. + + // lazy construction of a new set for faster comparisons + if (libraries.size() != requiredLibraries.size() || + !new HashSet<>(requiredLibraries).containsAll(libraries)) { + throw new IllegalStateException( - "The library registration references a different set of libraries than previous registrations for this job."); + "The library registration references a different set of library BLOBs than" + + " previous registrations for this job:\nold:" + libraries.toString() + + "\nnew:" + requiredLibraries.toString()); } - + + // lazy construction of a new set with String representations of the URLs + if (classPaths.size() != requiredClasspaths.size() || + !requiredClasspaths.stream().map(URL::toString).collect(Collectors.toSet()) + .containsAll(classPaths)) { + + throw new IllegalStateException( + "The library registration references a different set of library BLOBs than" + + " previous registrations for this job:\nold:" + + classPaths.toString() + + "\nnew:" + requiredClasspaths.toString()); + } + this.referenceHolders.add(task); } - + public boolean unregister(ExecutionAttemptID task) { referenceHolders.remove(task); return referenceHolders.isEmpty(); } - - public int getNumberOfReferenceHolders() { + + int getNumberOfReferenceHolders() { return referenceHolders.size(); } @@ -343,5 +319,4 @@ void releaseClassLoader() { } } } - } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java index 8e14e5867263a..41eeb1826b7fa 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/FallbackLibraryCacheManager.java @@ -28,7 +28,7 @@ import java.util.Collection; public class FallbackLibraryCacheManager implements LibraryCacheManager { - + private static Logger LOG = LoggerFactory.getLogger(FallbackLibraryCacheManager.class); @Override @@ -40,10 +40,10 @@ public ClassLoader getClassLoader(JobID id) { public void registerJob(JobID id, Collection requiredJarFiles, Collection requiredClasspaths) { LOG.warn("FallbackLibraryCacheManager cannot download files associated with blob keys."); } - + @Override public void registerTask(JobID id, ExecutionAttemptID execution, Collection requiredJarFiles, - Collection requiredClasspaths) { + Collection requiredClasspaths) { LOG.warn("FallbackLibraryCacheManager cannot download files associated with blob keys."); } @@ -51,7 +51,7 @@ public void registerTask(JobID id, ExecutionAttemptID execution, Collection requiredJarFiles, Collection requiredClasspaths) - throws IOException; - + throws IOException; + /** - * Registers a job task execution with its required jar files and classpaths. The jar files are identified by their blob keys. + * Registers a job task execution with its required jar files and classpaths. The jar files are + * identified by their blob keys and downloaded for use by a {@link ClassLoader}. * * @param id job ID * @param requiredJarFiles collection of blob keys identifying the required jar files * @param requiredClasspaths collection of classpaths that are added to the user code class loader - * @throws IOException + * + * @throws IOException if any error occurs when retrieving the required jar files * * @see #unregisterTask(JobID, ExecutionAttemptID) counterpart of this method */ void registerTask(JobID id, ExecutionAttemptID execution, Collection requiredJarFiles, - Collection requiredClasspaths) throws IOException; + Collection requiredClasspaths) throws IOException; /** * Unregisters a job task execution from the library cache manager. @@ -88,9 +93,7 @@ void registerTask(JobID id, ExecutionAttemptID execution, Collection re void unregisterJob(JobID id); /** - * Shutdown method - * - * @throws IOException + * Shutdown method which may release created class loaders. */ - void shutdown() throws IOException; + void shutdown(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java index 6b92d796d8709..c12687506524c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java @@ -536,7 +536,7 @@ public void uploadUserJars( Configuration blobClientConfig) throws IOException { if (!userJars.isEmpty()) { // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager - List blobKeys = BlobClient.uploadJarFiles(blobServerAddress, blobClientConfig, userJars); + List blobKeys = BlobClient.uploadJarFiles(blobServerAddress, blobClientConfig, jobID, userJars); for (BlobKey blobKey : blobKeys) { if (!userJarBlobKeys.contains(blobKey)) { 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 5838cf27461b8..c312cd361e932 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 @@ -21,7 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobService; +import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; @@ -93,7 +93,7 @@ public JobManagerRunner( final Configuration configuration, final RpcService rpcService, final HighAvailabilityServices haServices, - final BlobService blobService, + final BlobServer blobService, final HeartbeatServices heartbeatServices, final OnCompletionActions toNotifyOnComplete, final FatalErrorHandler errorHandler) throws Exception { @@ -116,7 +116,7 @@ public JobManagerRunner( final Configuration configuration, final RpcService rpcService, final HighAvailabilityServices haServices, - final BlobService blobService, + final BlobServer blobService, final HeartbeatServices heartbeatServices, final MetricRegistry metricRegistry, final OnCompletionActions toNotifyOnComplete, @@ -199,6 +199,7 @@ public JobManagerRunner( haServices, heartbeatServices, jobManagerServices.executorService, + jobManagerServices.blobServer, jobManagerServices.libraryCacheManager, jobManagerServices.restartStrategyFactory, jobManagerServices.rpcAskTimeout, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java index e14f5aff0a824..57aeaff5c7844 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerServices.java @@ -19,11 +19,10 @@ package org.apache.flink.runtime.jobmaster; 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.IllegalConfigurationException; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.blob.BlobService; +import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory; import org.apache.flink.runtime.util.ExecutorThreadFactory; @@ -45,6 +44,7 @@ public class JobManagerServices { public final ScheduledExecutorService executorService; + public final BlobServer blobServer; public final BlobLibraryCacheManager libraryCacheManager; public final RestartStrategyFactory restartStrategyFactory; @@ -53,11 +53,13 @@ public class JobManagerServices { public JobManagerServices( ScheduledExecutorService executorService, + BlobServer blobServer, BlobLibraryCacheManager libraryCacheManager, RestartStrategyFactory restartStrategyFactory, Time rpcAskTimeout) { this.executorService = checkNotNull(executorService); + this.blobServer = checkNotNull(blobServer); this.libraryCacheManager = checkNotNull(libraryCacheManager); this.restartStrategyFactory = checkNotNull(restartStrategyFactory); this.rpcAskTimeout = checkNotNull(rpcAskTimeout); @@ -80,8 +82,9 @@ public void shutdown() throws Exception { firstException = t; } + libraryCacheManager.shutdown(); try { - libraryCacheManager.shutdown(); + blobServer.close(); } catch (Throwable t) { if (firstException == null) { @@ -103,16 +106,12 @@ public void shutdown() throws Exception { public static JobManagerServices fromConfiguration( Configuration config, - BlobService blobService) throws Exception { + BlobServer blobServer) throws Exception { Preconditions.checkNotNull(config); - Preconditions.checkNotNull(blobService); + Preconditions.checkNotNull(blobServer); - final long cleanupInterval = config.getLong( - ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, - ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000; - - final BlobLibraryCacheManager libraryCacheManager = new BlobLibraryCacheManager(blobService, cleanupInterval); + final BlobLibraryCacheManager libraryCacheManager = new BlobLibraryCacheManager(blobServer); final FiniteDuration timeout; try { @@ -127,6 +126,7 @@ public static JobManagerServices fromConfiguration( return new JobManagerServices( futureExecutor, + blobServer, libraryCacheManager, RestartStrategyFactory.createRestartStrategyFactory(config), Time.of(timeout.length(), timeout.unit())); 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 d6019dbd317eb..a8a8632ccaca7 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 @@ -28,6 +28,7 @@ import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; @@ -149,7 +150,10 @@ public class JobMaster extends RpcEndpoint implements JobMasterGateway { /** Service to contend for and retrieve the leadership of JM and RM */ private final HighAvailabilityServices highAvailabilityServices; - /** Blob cache manager used across jobs */ + /** Blob server used across jobs */ + private final BlobServer blobServer; + + /** Blob library cache manager used across jobs */ private final BlobLibraryCacheManager libraryCacheManager; /** The metrics for the JobManager itself */ @@ -204,6 +208,7 @@ public JobMaster( HighAvailabilityServices highAvailabilityService, HeartbeatServices heartbeatServices, ScheduledExecutorService executor, + BlobServer blobServer, BlobLibraryCacheManager libraryCacheManager, RestartStrategyFactory restartStrategyFactory, Time rpcAskTimeout, @@ -221,6 +226,7 @@ public JobMaster( this.configuration = checkNotNull(configuration); this.rpcTimeout = rpcAskTimeout; this.highAvailabilityServices = checkNotNull(highAvailabilityService); + this.blobServer = checkNotNull(blobServer); this.libraryCacheManager = checkNotNull(libraryCacheManager); this.executor = checkNotNull(executor); this.jobCompletionActions = checkNotNull(jobCompletionActions); @@ -698,7 +704,7 @@ public void notifyKvStateUnregistered( @Override public CompletableFuture requestClassloadingProps() { return CompletableFuture.completedFuture( - new ClassloadingProps(libraryCacheManager.getBlobServerPort(), + new ClassloadingProps(blobServer.getPort(), executionGraph.getRequiredJarFiles(), executionGraph.getRequiredClasspaths())); } @@ -785,7 +791,7 @@ public CompletableFuture registerTaskManager( if (registeredTaskManagers.containsKey(taskManagerId)) { final RegistrationResponse response = new JMTMRegistrationSuccess( - resourceId, libraryCacheManager.getBlobServerPort()); + resourceId, blobServer.getPort()); return CompletableFuture.completedFuture(response); } else { return getRpcService() @@ -819,7 +825,7 @@ public void requestHeartbeat(ResourceID resourceID, Void payload) { } }); - return new JMTMRegistrationSuccess(resourceId, libraryCacheManager.getBlobServerPort()); + return new JMTMRegistrationSuccess(resourceId, blobServer.getPort()); }, getMainThreadExecutor()); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java index 98c7bf11d8f22..363c1072ef629 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.taskexecutor; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker; @@ -53,6 +54,9 @@ public class JobManagerConnection { // Checkpoint responder for the specific job manager private final CheckpointResponder checkpointResponder; + // BLOB cache connected to the BLOB server at the specific job manager + private final BlobCache blobCache; + // Library cache manager connected to the specific job manager private final LibraryCacheManager libraryCacheManager; @@ -63,21 +67,22 @@ public class JobManagerConnection { private final PartitionProducerStateChecker partitionStateChecker; public JobManagerConnection( - JobID jobID, - ResourceID resourceID, - JobMasterGateway jobMasterGateway, - UUID leaderId, - TaskManagerActions taskManagerActions, - CheckpointResponder checkpointResponder, - LibraryCacheManager libraryCacheManager, - ResultPartitionConsumableNotifier resultPartitionConsumableNotifier, - PartitionProducerStateChecker partitionStateChecker) { + JobID jobID, + ResourceID resourceID, + JobMasterGateway jobMasterGateway, + UUID leaderId, + TaskManagerActions taskManagerActions, + CheckpointResponder checkpointResponder, + BlobCache blobCache, LibraryCacheManager libraryCacheManager, + ResultPartitionConsumableNotifier resultPartitionConsumableNotifier, + PartitionProducerStateChecker partitionStateChecker) { this.jobID = Preconditions.checkNotNull(jobID); this.resourceID = Preconditions.checkNotNull(resourceID); this.leaderId = Preconditions.checkNotNull(leaderId); this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); this.taskManagerActions = Preconditions.checkNotNull(taskManagerActions); this.checkpointResponder = Preconditions.checkNotNull(checkpointResponder); + this.blobCache = Preconditions.checkNotNull(blobCache); this.libraryCacheManager = Preconditions.checkNotNull(libraryCacheManager); this.resultPartitionConsumableNotifier = Preconditions.checkNotNull(resultPartitionConsumableNotifier); this.partitionStateChecker = Preconditions.checkNotNull(partitionStateChecker); @@ -111,6 +116,15 @@ public LibraryCacheManager getLibraryCacheManager() { return libraryCacheManager; } + /** + * Gets the BLOB cache connected to the respective BLOB server instance at the job manager. + * + * @return BLOB cache + */ + public BlobCache getBlobCache() { + return blobCache; + } + public ResultPartitionConsumableNotifier getResultPartitionConsumableNotifier() { return resultPartitionConsumableNotifier; } 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 4abcdf4365d3b..a5ce84be5cdcd 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 @@ -352,6 +352,7 @@ public CompletableFuture submitTask( TaskManagerActions taskManagerActions = jobManagerConnection.getTaskManagerActions(); CheckpointResponder checkpointResponder = jobManagerConnection.getCheckpointResponder(); + BlobCache blobCache = jobManagerConnection.getBlobCache(); LibraryCacheManager libraryCache = jobManagerConnection.getLibraryCacheManager(); ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = jobManagerConnection.getResultPartitionConsumableNotifier(); PartitionProducerStateChecker partitionStateChecker = jobManagerConnection.getPartitionStateChecker(); @@ -374,6 +375,7 @@ public CompletableFuture submitTask( taskManagerActions, inputSplitProvider, checkpointResponder, + blobCache, libraryCache, fileCache, taskManagerConfiguration, @@ -935,14 +937,13 @@ private JobManagerConnection associateWithJobManager( InetSocketAddress blobServerAddress = new InetSocketAddress(jobMasterGateway.getHostname(), blobPort); final LibraryCacheManager libraryCacheManager; + final BlobCache blobCache; try { - final BlobCache blobCache = new BlobCache( + blobCache = new BlobCache( blobServerAddress, taskManagerConfiguration.getConfiguration(), haServices.createBlobStore()); - libraryCacheManager = new BlobLibraryCacheManager( - blobCache, - taskManagerConfiguration.getCleanupInterval()); + libraryCacheManager = new BlobLibraryCacheManager(blobCache); } catch (IOException e) { // Can't pass the IOException up - we need a RuntimeException anyway // two levels up where this is run asynchronously. Also, we don't @@ -967,6 +968,7 @@ private JobManagerConnection associateWithJobManager( jobManagerLeaderId, taskManagerActions, checkpointResponder, + blobCache, libraryCacheManager, resultPartitionConsumableNotifier, partitionStateChecker); @@ -977,6 +979,7 @@ private void disassociateFromJobManager(JobManagerConnection jobManagerConnectio JobMasterGateway jobManagerGateway = jobManagerConnection.getJobManagerGateway(); jobManagerGateway.disconnectTaskManager(getResourceID(), cause); jobManagerConnection.getLibraryCacheManager().shutdown(); + jobManagerConnection.getBlobCache().close(); } // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java index ea9f5767b0157..7c7693bb9a1b0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerConfiguration.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; @@ -53,8 +54,6 @@ public class TaskManagerConfiguration implements TaskManagerRuntimeInfo { private final Time maxRegistrationPause; private final Time refusedRegistrationPause; - private final long cleanupInterval; - private final UnmodifiableConfiguration configuration; private final boolean exitJvmOnOutOfMemory; @@ -78,7 +77,6 @@ public TaskManagerConfiguration( this.initialRegistrationPause = Preconditions.checkNotNull(initialRegistrationPause); this.maxRegistrationPause = Preconditions.checkNotNull(maxRegistrationPause); this.refusedRegistrationPause = Preconditions.checkNotNull(refusedRegistrationPause); - this.cleanupInterval = Preconditions.checkNotNull(cleanupInterval); this.configuration = new UnmodifiableConfiguration(Preconditions.checkNotNull(configuration)); this.exitJvmOnOutOfMemory = exitJvmOnOutOfMemory; } @@ -107,10 +105,6 @@ public Time getRefusedRegistrationPause() { return refusedRegistrationPause; } - public long getCleanupInterval() { - return cleanupInterval; - } - @Override public Configuration getConfiguration() { return configuration; @@ -153,9 +147,7 @@ public static TaskManagerConfiguration fromConfiguration(Configuration configura LOG.info("Messages have a max timeout of " + timeout); - final long cleanupInterval = configuration.getLong( - ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, - ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000; + final long cleanupInterval = configuration.getLong(BlobServerOptions.CLEANUP_INTERVAL) * 1000; final Time finiteRegistrationDuration; 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 04cb99038dc4e..d62896054d40c 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 @@ -30,6 +30,7 @@ import org.apache.flink.core.fs.SafetyNetCloseableRegistry; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.accumulators.AccumulatorRegistry; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; @@ -203,7 +204,10 @@ public class Task implements Runnable, TaskActions { /** All listener that want to be notified about changes in the task's execution state */ private final List taskExecutionStateListeners; - /** The library cache, from which the task can request its required JAR files */ + /** The BLOB cache, from which the task can request BLOB files */ + private final BlobCache blobCache; + + /** 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 */ @@ -282,6 +286,7 @@ public Task( TaskManagerActions taskManagerActions, InputSplitProvider inputSplitProvider, CheckpointResponder checkpointResponder, + BlobCache blobCache, LibraryCacheManager libraryCache, FileCache fileCache, TaskManagerRuntimeInfo taskManagerConfig, @@ -330,6 +335,7 @@ public Task( this.checkpointResponder = Preconditions.checkNotNull(checkpointResponder); this.taskManagerActions = checkNotNull(taskManagerActions); + this.blobCache = Preconditions.checkNotNull(blobCache); this.libraryCache = Preconditions.checkNotNull(libraryCache); this.fileCache = Preconditions.checkNotNull(fileCache); this.network = Preconditions.checkNotNull(networkEnvironment); @@ -568,6 +574,8 @@ else if (current == ExecutionState.CANCELING) { LOG.info("Creating FileSystem stream leak safety net for task {}", this); FileSystemSafetyNet.initializeSafetyNetForThread(); + blobCache.registerJob(jobId); + // first of all, get a user-code classloader // this may involve downloading the job's JAR files and/or classes LOG.info("Loading JAR files for task {}.", this); @@ -827,6 +835,7 @@ else if (transitionState(current, ExecutionState.FAILED, t)) { // remove all of the tasks library resources libraryCache.unregisterTask(jobId, executionId); + blobCache.releaseJob(jobId); // remove all files in the distributed cache removeCachedFiles(distributedCacheEntries, fileCache); @@ -862,7 +871,7 @@ private ClassLoader createUserCodeClassloader(LibraryCacheManager libraryCache) // triggers the download of all missing jar files from the job manager libraryCache.registerTask(jobId, executionId, requiredJarFiles, requiredClasspaths); - LOG.debug("Register task {} at library cache manager took {} milliseconds", + LOG.debug("Getting user code class loader for task {} at library cache manager took {} milliseconds", executionId, System.currentTimeMillis() - startDownloadTime); ClassLoader userCodeClassLoader = libraryCache.getClassLoader(jobId); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala index cd7b363924a2d..61c61b4c0ee4c 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala @@ -18,11 +18,12 @@ package org.apache.flink.runtime.clusterframework -import java.util.concurrent.{ScheduledExecutorService, Executor} +import java.util.concurrent.{Executor, ScheduledExecutorService} import akka.actor.ActorRef import org.apache.flink.api.common.JobID import org.apache.flink.configuration.Configuration +import org.apache.flink.runtime.blob.BlobServer import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.clusterframework.messages._ import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager @@ -51,6 +52,7 @@ import scala.language.postfixOps * @param instanceManager Instance manager to manage the registered * [[org.apache.flink.runtime.taskmanager.TaskManager]] * @param scheduler Scheduler to schedule Flink jobs + * @param blobServer Server instance to store BLOBs for the individual tasks * @param libraryCacheManager Manager to manage uploaded jar files * @param archive Archive for finished Flink jobs * @param restartStrategyFactory Restart strategy to be used in case of a job recovery @@ -63,6 +65,7 @@ abstract class ContaineredJobManager( ioExecutor: Executor, instanceManager: InstanceManager, scheduler: FlinkScheduler, + blobServer: BlobServer, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, restartStrategyFactory: RestartStrategyFactory, @@ -78,6 +81,7 @@ abstract class ContaineredJobManager( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 1616a7b7b6663..8c551a730c9f6 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -126,6 +126,7 @@ class JobManager( protected val ioExecutor: Executor, protected val instanceManager: InstanceManager, protected val scheduler: FlinkScheduler, + protected val blobServer: BlobServer, protected val libraryCacheManager: BlobLibraryCacheManager, protected val archive: ActorRef, protected val restartStrategyFactory: RestartStrategyFactory, @@ -272,11 +273,12 @@ class JobManager( instanceManager.shutdown() scheduler.shutdown() + libraryCacheManager.shutdown() try { - libraryCacheManager.shutdown() + blobServer.close() } catch { - case e: IOException => log.error("Could not properly shutdown the library cache manager.", e) + case e: IOException => log.error("Could not properly shutdown the blob server.", e) } // failsafe shutdown of the metrics registry @@ -422,7 +424,7 @@ class JobManager( taskManager ! decorateMessage( AlreadyRegistered( instanceID, - libraryCacheManager.getBlobServerPort)) + blobServer.getPort)) } else { try { val actorGateway = new AkkaActorGateway(taskManager, leaderSessionID.orNull) @@ -437,7 +439,7 @@ class JobManager( taskManagerMap.put(taskManager, instanceID) taskManager ! decorateMessage( - AcknowledgeRegistration(instanceID, libraryCacheManager.getBlobServerPort)) + AcknowledgeRegistration(instanceID, blobServer.getPort)) // to be notified when the taskManager is no longer reachable context.watch(taskManager) @@ -839,6 +841,7 @@ class JobManager( try { log.info(s"Disposing savepoint at '$savepointPath'.") //TODO user code class loader ? + // (has not been used so far and new savepoints can simply be deleted by file) val savepoint = SavepointStore.loadSavepoint( savepointPath, Thread.currentThread().getContextClassLoader) @@ -1060,7 +1063,7 @@ class JobManager( case Some((graph, jobInfo)) => sender() ! decorateMessage( ClassloadingProps( - libraryCacheManager.getBlobServerPort, + blobServer.getPort, graph.getRequiredJarFiles, graph.getRequiredClasspaths)) case None => @@ -1068,7 +1071,7 @@ class JobManager( } case RequestBlobManagerPort => - sender ! decorateMessage(libraryCacheManager.getBlobServerPort) + sender ! decorateMessage(blobServer.getPort) case RequestArchive => sender ! decorateMessage(ResponseArchive(archive)) @@ -1254,8 +1257,8 @@ class JobManager( // because this makes sure that the uploaded jar files are removed in case of // unsuccessful try { - libraryCacheManager.registerJob(jobGraph.getJobID, jobGraph.getUserJarBlobKeys, - jobGraph.getClasspaths) + libraryCacheManager.registerJob( + jobGraph.getJobID, jobGraph.getUserJarBlobKeys, jobGraph.getClasspaths) } catch { case t: Throwable => @@ -1344,6 +1347,7 @@ class JobManager( log.error(s"Failed to submit job $jobId ($jobName)", t) libraryCacheManager.unregisterJob(jobId) + blobServer.cleanupJob(jobId) currentJobs.remove(jobId) if (executionGraph != null) { @@ -1785,12 +1789,10 @@ class JobManager( case None => None } - try { - libraryCacheManager.unregisterJob(jobID) - } catch { - case t: Throwable => - log.error(s"Could not properly unregister job $jobID from the library cache.", t) - } + // remove all job-related BLOBs from local and HA store + libraryCacheManager.unregisterJob(jobID) + blobServer.cleanupJob(jobID) + jobManagerMetricGroup.foreach(_.removeJob(jobID)) futureOption @@ -2463,6 +2465,7 @@ object JobManager { blobStore: BlobStore) : (InstanceManager, FlinkScheduler, + BlobServer, BlobLibraryCacheManager, RestartStrategyFactory, FiniteDuration, // timeout @@ -2474,10 +2477,6 @@ object JobManager { val timeout: FiniteDuration = AkkaUtils.getTimeout(configuration) - val cleanupInterval = configuration.getLong( - ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL, - ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000 - val restartStrategy = RestartStrategyFactory.createRestartStrategyFactory(configuration) val archiveCount = configuration.getInteger(WebOptions.ARCHIVE_COUNT) @@ -2508,21 +2507,21 @@ object JobManager { blobServer = new BlobServer(configuration, blobStore) instanceManager = new InstanceManager() scheduler = new FlinkScheduler(ExecutionContext.fromExecutor(futureExecutor)) - libraryCacheManager = new BlobLibraryCacheManager(blobServer, cleanupInterval) + libraryCacheManager = new BlobLibraryCacheManager(blobServer) instanceManager.addInstanceListener(scheduler) } catch { case t: Throwable => - if (libraryCacheManager != null) { - libraryCacheManager.shutdown() - } if (scheduler != null) { scheduler.shutdown() } if (instanceManager != null) { instanceManager.shutdown() } + if (libraryCacheManager != null) { + libraryCacheManager.shutdown() + } if (blobServer != null) { blobServer.close() } @@ -2554,6 +2553,7 @@ object JobManager { (instanceManager, scheduler, + blobServer, libraryCacheManager, restartStrategy, timeout, @@ -2627,6 +2627,7 @@ object JobManager { val (instanceManager, scheduler, + blobServer, libraryCacheManager, restartStrategy, timeout, @@ -2654,6 +2655,7 @@ object JobManager { ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategy, @@ -2693,6 +2695,7 @@ object JobManager { ioExecutor: Executor, instanceManager: InstanceManager, scheduler: FlinkScheduler, + blobServer: BlobServer, libraryCacheManager: LibraryCacheManager, archive: ActorRef, restartStrategyFactory: RestartStrategyFactory, @@ -2710,6 +2713,7 @@ object JobManager { ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala index 0ae00a93852cf..dcf9dd0b80e62 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/LocalFlinkMiniCluster.scala @@ -26,6 +26,7 @@ import org.apache.flink.api.common.JobID import org.apache.flink.api.common.io.FileOutputFormat import org.apache.flink.configuration.{ConfigConstants, Configuration, JobManagerOptions, QueryableStateOptions, ResourceManagerOptions, TaskManagerOptions} import org.apache.flink.core.fs.Path +import org.apache.flink.runtime.blob.BlobServer import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.clusterframework.FlinkResourceManager import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager @@ -133,6 +134,7 @@ class LocalFlinkMiniCluster( val (instanceManager, scheduler, + blobServer, libraryCacheManager, restartStrategyFactory, timeout, @@ -164,6 +166,7 @@ class LocalFlinkMiniCluster( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, @@ -279,6 +282,7 @@ class LocalFlinkMiniCluster( ioExecutor: Executor, instanceManager: InstanceManager, scheduler: Scheduler, + blobServer: BlobServer, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, restartStrategyFactory: RestartStrategyFactory, @@ -297,6 +301,7 @@ class LocalFlinkMiniCluster( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 0c419eb460f13..431adb6f8b122 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -35,7 +35,7 @@ import org.apache.flink.configuration._ import org.apache.flink.core.fs.FileSystem import org.apache.flink.runtime.accumulators.AccumulatorSnapshot import org.apache.flink.runtime.akka.{AkkaUtils, DefaultQuarantineHandler, QuarantineMonitor} -import org.apache.flink.runtime.blob.{BlobCache, BlobClient, BlobService} +import org.apache.flink.runtime.blob.{BlobCache, BlobClient} import org.apache.flink.runtime.broadcast.BroadcastVariableManager import org.apache.flink.runtime.clusterframework.messages.StopCluster import org.apache.flink.runtime.clusterframework.types.ResourceID @@ -160,7 +160,7 @@ class TaskManager( * registered at the job manager */ private val waitForRegistration = scala.collection.mutable.Set[ActorRef]() - private var blobService: Option[BlobService] = None + private var blobCache: Option[BlobCache] = None private var libraryCacheManager: Option[LibraryCacheManager] = None /* The current leading JobManager Actor associated with */ @@ -333,11 +333,11 @@ class TaskManager( killTaskManagerFatal(message, cause) case RequestTaskManagerLog(requestType : LogTypeRequest) => - blobService match { + blobCache match { case Some(_) => handleRequestTaskManagerLog(sender(), requestType, currentJobManager.get) case None => - sender() ! akka.actor.Status.Failure(new IOException("BlobService not " + + sender() ! akka.actor.Status.Failure(new IOException("BlobCache not " + "available. Cannot upload TaskManager logs.")) } @@ -840,7 +840,7 @@ class TaskManager( if (file.exists()) { val fis = new FileInputStream(file); Future { - val client: BlobClient = blobService.get.createClient() + val client: BlobClient = blobCache.get.createClient() client.put(fis); }(context.dispatcher) .onComplete { @@ -915,7 +915,7 @@ class TaskManager( "starting network stack and library cache.") // sanity check that the JobManager dependent components are not set up currently - if (connectionUtils.isDefined || blobService.isDefined) { + if (connectionUtils.isDefined || blobCache.isDefined) { throw new IllegalStateException("JobManager-specific components are already initialized.") } @@ -968,9 +968,9 @@ class TaskManager( address, config.getConfiguration(), highAvailabilityServices.createBlobStore()) - blobService = Option(blobcache) + blobCache = Option(blobcache) libraryCacheManager = Some( - new BlobLibraryCacheManager(blobcache, config.getCleanupInterval())) + new BlobLibraryCacheManager(blobcache)) } catch { case e: Exception => @@ -1047,18 +1047,11 @@ class TaskManager( // shut down BLOB and library cache libraryCacheManager foreach { - manager => - try { - manager.shutdown() - } catch { - case ioe: IOException => log.error( - "Could not properly shutdown library cache manager.", - ioe) - } + manager => manager.shutdown() } libraryCacheManager = None - blobService foreach { + blobCache foreach { service => try { service.close() @@ -1066,7 +1059,7 @@ class TaskManager( case ioe: IOException => log.error("Could not properly shutdown blob service.", ioe) } } - blobService = None + blobCache = None // disassociate the slot environment connectionUtils = None @@ -1130,6 +1123,10 @@ class TaskManager( case Some(manager) => manager case None => throw new IllegalStateException("There is no valid library cache manager.") } + val blobCache = this.blobCache match { + case Some(manager) => manager + case None => throw new IllegalStateException("There is no valid BLOB cache.") + } val slot = tdd.getTargetSlotNumber if (slot < 0 || slot >= numberOfSlots) { @@ -1200,6 +1197,7 @@ class TaskManager( taskManagerConnection, inputSplitProvider, checkpointResponder, + blobCache, libCache, fileCache, config, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java new file mode 100644 index 0000000000000..afd365b6fc892 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java @@ -0,0 +1,328 @@ +/* + * 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.blob; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.TestLogger; + +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +/** + * A few tests for the deferred ref-counting based cleanup inside the {@link BlobCache}. + */ +public class BlobCacheCleanupTest extends TestLogger { + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + /** + * Tests that {@link BlobCache} cleans up after calling {@link BlobCache#releaseJob(JobID)}. + */ + @Test + public void testJobCleanup() throws IOException, InterruptedException { + + JobID jobId = new JobID(); + List keys = new ArrayList(); + BlobServer server = null; + BlobCache cache = null; + + final byte[] buf = new byte[128]; + + try { + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, 1L); + + server = new BlobServer(config, new VoidBlobStore()); + InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort()); + + // upload blobs + try (BlobClient bc = new BlobClient(serverAddress, config)) { + keys.add(bc.put(jobId, buf)); + buf[0] += 1; + keys.add(bc.put(jobId, buf)); + } + + cache = new BlobCache(serverAddress, config, new VoidBlobStore()); + + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(0, jobId, cache); + + // register once + cache.registerJob(jobId); + + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(0, jobId, cache); + + for (BlobKey key : keys) { + cache.getFile(jobId, key); + } + + // register again (let's say, from another thread or so) + cache.registerJob(jobId); + for (BlobKey key : keys) { + cache.getFile(jobId, key); + } + + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + // after releasing once, nothing should change + cache.releaseJob(jobId); + + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + // after releasing the second time, the job is up for deferred cleanup + cache.releaseJob(jobId); + + // because we cannot guarantee that there are not thread races in the build system, we + // loop for a certain while until the references disappear + { + long deadline = System.currentTimeMillis() + 30_000L; + do { + Thread.sleep(100); + } + while (checkFilesExist(jobId, keys, cache, false) != 0 && + System.currentTimeMillis() < deadline); + } + + // the blob cache should no longer contain the files + // this fails if we exited via a timeout + checkFileCountForJob(0, jobId, cache); + // server should be unaffected + checkFileCountForJob(2, jobId, server); + } + finally { + if (cache != null) { + cache.close(); + } + + if (server != null) { + server.close(); + } + // now everything should be cleaned up + checkFileCountForJob(0, jobId, server); + } + } + + /** + * Tests that {@link BlobCache} cleans up after calling {@link BlobCache#releaseJob(JobID)} + * but only after preserving the file for a bit longer. + */ + @Test + @Ignore("manual test due to stalling: ensures a BLOB is retained first and only deleted after the (long) timeout ") + public void testJobDeferredCleanup() throws IOException, InterruptedException { + // file should be deleted between 5 and 10s after last job release + long cleanupInterval = 5L; + + JobID jobId = new JobID(); + List keys = new ArrayList(); + BlobServer server = null; + BlobCache cache = null; + + final byte[] buf = new byte[128]; + + try { + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); + + server = new BlobServer(config, new VoidBlobStore()); + InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort()); + + // upload blobs + try (BlobClient bc = new BlobClient(serverAddress, config)) { + keys.add(bc.put(jobId, buf)); + buf[0] += 1; + keys.add(bc.put(jobId, buf)); + } + + cache = new BlobCache(serverAddress, config, new VoidBlobStore()); + + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(0, jobId, cache); + + // register once + cache.registerJob(jobId); + + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(0, jobId, cache); + + for (BlobKey key : keys) { + cache.getFile(jobId, key); + } + + // register again (let's say, from another thread or so) + cache.registerJob(jobId); + for (BlobKey key : keys) { + cache.getFile(jobId, key); + } + + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + // after releasing once, nothing should change + cache.releaseJob(jobId); + + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + // after releasing the second time, the job is up for deferred cleanup + cache.releaseJob(jobId); + + // files should still be accessible for now + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, cache); + + Thread.sleep(cleanupInterval / 5); + // still accessible... + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, cache); + + Thread.sleep((cleanupInterval * 4) / 5); + + // files are up for cleanup now...wait for it: + // because we cannot guarantee that there are not thread races in the build system, we + // loop for a certain while until the references disappear + { + long deadline = System.currentTimeMillis() + 30_000L; + do { + Thread.sleep(100); + } + while (checkFilesExist(jobId, keys, cache, false) != 0 && + System.currentTimeMillis() < deadline); + } + + // the blob cache should no longer contain the files + // this fails if we exited via a timeout + checkFileCountForJob(0, jobId, cache); + // server should be unaffected + checkFileCountForJob(2, jobId, server); + } + finally { + if (cache != null) { + cache.close(); + } + + if (server != null) { + server.close(); + } + // now everything should be cleaned up + checkFileCountForJob(0, jobId, server); + } + } + + /** + * Checks how many of the files given by blob keys are accessible. + * + * @param jobId + * ID of a job + * @param keys + * blob keys to check + * @param blobService + * BLOB store to use + * @param doThrow + * whether exceptions should be ignored (false), or thrown (true) + * + * @return number of files we were able to retrieve via {@link BlobService#getFile} + */ + public static int checkFilesExist( + JobID jobId, Collection keys, BlobService blobService, boolean doThrow) + throws IOException { + + int numFiles = 0; + + for (BlobKey key : keys) { + final File blobFile; + if (blobService instanceof BlobServer) { + BlobServer server = (BlobServer) blobService; + blobFile = server.getStorageLocation(jobId, key); + } else { + BlobCache cache = (BlobCache) blobService; + blobFile = cache.getStorageLocation(jobId, key); + } + if (blobFile.exists()) { + ++numFiles; + } else if (doThrow) { + throw new IOException("File " + blobFile + " does not exist."); + } + } + + return numFiles; + } + + /** + * Checks how many of the files given by blob keys are accessible. + * + * @param expectedCount + * number of expected files in the blob service for the given job + * @param jobId + * ID of a job + * @param blobService + * BLOB store to use + * + * @return number of files we were able to retrieve via {@link BlobService#getFile} + */ + public static void checkFileCountForJob( + int expectedCount, JobID jobId, BlobService blobService) + throws IOException { + + final File jobDir; + if (blobService instanceof BlobServer) { + BlobServer server = (BlobServer) blobService; + jobDir = server.getStorageLocation(jobId, new BlobKey()).getParentFile(); + } else { + BlobCache cache = (BlobCache) blobService; + jobDir = cache.getStorageLocation(jobId, new BlobKey()).getParentFile(); + } + File[] blobsForJob = jobDir.listFiles(); + if (blobsForJob == null) { + if (expectedCount != 0) { + throw new IOException("File " + jobDir + " does not exist."); + } + } else { + assertEquals("Too many/few files in job dir: " + + Arrays.asList(blobsForJob).toString(), expectedCount, + blobsForJob.length); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java index 8c575a959b9e5..0060ccbf08fda 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheRetriesTest.java @@ -22,6 +22,8 @@ import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.util.TestLogger; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -37,7 +39,7 @@ /** * Unit tests for the blob cache retrying the connection to the server. */ -public class BlobCacheRetriesTest { +public class BlobCacheRetriesTest extends TestLogger { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java index d511e86cfeacf..6d6bfd51f5dc0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobClientTest.java @@ -39,6 +39,8 @@ import java.util.List; import org.apache.flink.api.common.JobID; +import org.apache.flink.util.TestLogger; + import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -46,7 +48,7 @@ /** * This class contains unit tests for the {@link BlobClient}. */ -public class BlobClientTest { +public class BlobClientTest extends TestLogger { /** The buffer size used during the tests in bytes. */ private static final int TEST_BUFFER_SIZE = 17 * 1000; @@ -214,7 +216,7 @@ private static void validateGetAndClose(final InputStream inputStream, final Fil * Tests the PUT/GET operations for content-addressable buffers. */ @Test - public void testContentAddressableBuffer() { + public void testContentAddressableBuffer() throws IOException { BlobClient client = null; @@ -256,10 +258,6 @@ public void testContentAddressableBuffer() { // expected } } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } finally { if (client != null) { try { @@ -281,7 +279,7 @@ protected BlobServer getBlobServer() { * Tests the PUT/GET operations for content-addressable streams. */ @Test - public void testContentAddressableStream() { + public void testContentAddressableStream() throws IOException { BlobClient client = null; InputStream is = null; @@ -313,10 +311,6 @@ public void testContentAddressableStream() { validateGetAndClose(client.get(receivedKey), testFile); validateGetAndClose(client.get(jobId, receivedKey), testFile); } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } finally { if (is != null) { try { @@ -332,7 +326,7 @@ public void testContentAddressableStream() { } /** - * Tests the static {@link BlobClient#uploadJarFiles(InetSocketAddress, Configuration, List)} helper. + * Tests the static {@link BlobClient#uploadJarFiles(InetSocketAddress, Configuration, JobID, List)} helper. */ @Test public void testUploadJarFilesHelper() throws Exception { @@ -340,7 +334,7 @@ public void testUploadJarFilesHelper() throws Exception { } /** - * Tests the static {@link BlobClient#uploadJarFiles(InetSocketAddress, Configuration, List)} helper. + * Tests the static {@link BlobClient#uploadJarFiles(InetSocketAddress, Configuration, JobID, List)}} helper. */ static void uploadJarFile(BlobServer blobServer, Configuration blobClientConfig) throws Exception { final File testFile = File.createTempFile("testfile", ".dat"); @@ -354,15 +348,16 @@ static void uploadJarFile(BlobServer blobServer, Configuration blobClientConfig) } private static void uploadJarFile( - final InetSocketAddress serverAddress, final Configuration blobClientConfig, - final File testFile) throws IOException { + final InetSocketAddress serverAddress, final Configuration blobClientConfig, + final File testFile) throws IOException { + JobID jobId = new JobID(); List blobKeys = BlobClient.uploadJarFiles(serverAddress, blobClientConfig, - Collections.singletonList(new Path(testFile.toURI()))); + jobId, Collections.singletonList(new Path(testFile.toURI()))); assertEquals(1, blobKeys.size()); try (BlobClient blobClient = new BlobClient(serverAddress, blobClientConfig)) { - validateGetAndClose(blobClient.get(blobKeys.get(0)), testFile); + validateGetAndClose(blobClient.get(jobId, blobKeys.get(0)), testFile); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java index 4071a1caf5544..43bc6228248f2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobKeyTest.java @@ -29,12 +29,14 @@ import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.util.StringUtils; +import org.apache.flink.util.TestLogger; + import org.junit.Test; /** * This class contains unit tests for the {@link BlobKey} class. */ -public final class BlobKeyTest { +public final class BlobKeyTest extends TestLogger { /** * The first key array to be used during the unit tests. */ @@ -106,4 +108,4 @@ public void testStreams() throws Exception { assertEquals(k1, k2); } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java index 413e2e9f2ed78..6bb5ab57416ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java @@ -41,6 +41,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFileCountForJob; +import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFilesExist; import static org.apache.flink.runtime.blob.BlobClientTest.validateGetAndClose; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -62,7 +64,7 @@ public class BlobServerDeleteTest extends TestLogger { public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Test - public void testDeleteSingleByBlobKey() { + public void testDeleteSingleByBlobKey() throws IOException { BlobServer server = null; BlobClient client = null; BlobStore blobStore = new VoidBlobStore(); @@ -131,10 +133,6 @@ public void testDeleteSingleByBlobKey() { // expected } } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } finally { cleanup(server, client); } @@ -153,16 +151,16 @@ private static void ensureClientIsClosed(final BlobClient client) throws IOExcep } @Test - public void testDeleteAlreadyDeletedNoJob() { + public void testDeleteAlreadyDeletedNoJob() throws IOException { testDeleteAlreadyDeleted(null); } @Test - public void testDeleteAlreadyDeletedForJob() { + public void testDeleteAlreadyDeletedForJob() throws IOException { testDeleteAlreadyDeleted(new JobID()); } - private void testDeleteAlreadyDeleted(final JobID jobId) { + private void testDeleteAlreadyDeleted(final JobID jobId) throws IOException { BlobServer server = null; BlobClient client = null; BlobStore blobStore = new VoidBlobStore(); @@ -201,10 +199,6 @@ private void testDeleteAlreadyDeleted(final JobID jobId) { server.delete(jobId, key); } } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); - } finally { cleanup(server, client); } @@ -219,16 +213,16 @@ private static void deleteHelper(BlobClient client, JobID jobId, BlobKey key) th } @Test - public void testDeleteFailsNoJob() { + public void testDeleteFailsNoJob() throws IOException { testDeleteFails(null); } @Test - public void testDeleteFailsForJob() { + public void testDeleteFailsForJob() throws IOException { testDeleteFails(new JobID()); } - private void testDeleteFails(final JobID jobId) { + private void testDeleteFails(final JobID jobId) throws IOException { assumeTrue(!OperatingSystem.isWindows()); //setWritable doesn't work on Windows. BlobServer server = null; @@ -275,9 +269,6 @@ private void testDeleteFails(final JobID jobId) { } else { server.getFile(jobId, key); } - } catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); } finally { if (blobFile != null && directory != null) { //noinspection ResultOfMethodCallIgnored @@ -289,6 +280,64 @@ private void testDeleteFails(final JobID jobId) { } } + /** + * Tests that {@link BlobServer} cleans up after calling {@link BlobServer#cleanupJob(JobID)}. + */ + @Test + public void testJobCleanup() throws IOException, InterruptedException { + + JobID jobId1 = new JobID(); + List keys1 = new ArrayList(); + JobID jobId2 = new JobID(); + List keys2 = new ArrayList(); + BlobServer server = null; + + final byte[] buf = new byte[128]; + + try { + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + + server = new BlobServer(config, new VoidBlobStore()); + InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort()); + BlobClient bc = new BlobClient(serverAddress, config); + + keys1.add(bc.put(jobId1, buf)); + keys2.add(bc.put(jobId2, buf)); + assertEquals(keys2.get(0), keys1.get(0)); + + buf[0] += 1; + keys1.add(bc.put(jobId1, buf)); + + bc.close(); + + assertEquals(2, checkFilesExist(jobId1, keys1, server, true)); + checkFileCountForJob(2, jobId1, server); + assertEquals(1, checkFilesExist(jobId2, keys2, server, true)); + checkFileCountForJob(1, jobId2, server); + + server.cleanupJob(jobId1); + + checkFileCountForJob(0, jobId1, server); + assertEquals(1, checkFilesExist(jobId2, keys2, server, true)); + checkFileCountForJob(1, jobId2, server); + + server.cleanupJob(jobId2); + + checkFileCountForJob(0, jobId1, server); + checkFileCountForJob(0, jobId2, server); + + // calling a second time should not fail + server.cleanupJob(jobId2); + } + finally { + if (server != null) { + server.close(); + } + } + } + /** * FLINK-6020 * diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java index e449aab822854..a6ac44790eeaa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobUtilsTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.util.OperatingSystem; +import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; @@ -34,7 +35,7 @@ import static org.junit.Assume.assumeTrue; import static org.mockito.Mockito.mock; -public class BlobUtilsTest { +public class BlobUtilsTest extends TestLogger { private final static String CANNOT_CREATE_THIS = "cannot-create-this"; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java index ec1bbd8e2157a..c58e3a0d0eee9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CoordinatorShutdownTest.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.testtasks.FailingBlockingInvokable; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -190,26 +191,4 @@ public static void unblock() { } } - public static class FailingBlockingInvokable extends AbstractInvokable { - private static boolean blocking = true; - private static final Object lock = new Object(); - - @Override - public void invoke() throws Exception { - while (blocking) { - synchronized (lock) { - lock.wait(); - } - } - throw new RuntimeException("This exception is expected."); - } - - public static void unblock() { - blocking = false; - - synchronized (lock) { - lock.notifyAll(); - } - } - } } 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 38146848f2369..423732710100d 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 @@ -22,7 +22,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -137,7 +136,7 @@ protected JobManagerRunner createJobManagerRunner( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - BlobService blobService, + BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, OnCompletionActions onCompleteActions, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java index b43a307447d83..a4b48e80cdd1b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheManagerTest.java @@ -18,24 +18,22 @@ package org.apache.flink.runtime.execution.librarycache; +import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobClient; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.api.common.JobID; import org.apache.flink.util.OperatingSystem; +import org.apache.flink.util.TestLogger; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import static org.junit.Assert.*; -import static org.junit.Assume.assumeTrue; - import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; @@ -45,7 +43,19 @@ import java.util.Collections; import java.util.List; -public class BlobLibraryCacheManagerTest { +import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFileCountForJob; +import static org.apache.flink.runtime.blob.BlobCacheCleanupTest.checkFilesExist; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.junit.Assume.assumeTrue; + +/** + * Tests for {@link BlobLibraryCacheManager}. + */ +public class BlobLibraryCacheManagerTest extends TestLogger { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -57,10 +67,13 @@ public class BlobLibraryCacheManagerTest { @Test public void testLibraryCacheManagerJobCleanup() throws IOException, InterruptedException { - JobID jid = new JobID(); - List keys = new ArrayList(); + JobID jobId1 = new JobID(); + JobID jobId2 = new JobID(); + List keys1 = new ArrayList<>(); + List keys2 = new ArrayList<>(); BlobServer server = null; - BlobLibraryCacheManager libraryCacheManager = null; + BlobCache cache = null; + BlobLibraryCacheManager libCache = null; final byte[] buf = new byte[128]; @@ -68,122 +81,231 @@ public void testLibraryCacheManagerJobCleanup() throws IOException, InterruptedE Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, 1L); server = new BlobServer(config, new VoidBlobStore()); - InetSocketAddress blobSocketAddress = new InetSocketAddress(server.getPort()); - BlobClient bc = new BlobClient(blobSocketAddress, config); - - // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager - JobID jobId = null; + InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort()); + BlobClient bc = new BlobClient(serverAddress, config); + cache = new BlobCache(serverAddress, config, new VoidBlobStore()); - keys.add(bc.put(jobId, buf)); + keys1.add(bc.put(jobId1, buf)); buf[0] += 1; - keys.add(bc.put(jobId, buf)); + keys1.add(bc.put(jobId1, buf)); + keys2.add(bc.put(jobId2, buf)); bc.close(); - long cleanupInterval = 1000L; - libraryCacheManager = new BlobLibraryCacheManager(server, cleanupInterval); - libraryCacheManager.registerJob(jid, keys, Collections.emptyList()); + libCache = new BlobLibraryCacheManager(cache); + cache.registerJob(jobId1); + cache.registerJob(jobId2); + + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId1)); + checkFileCountForJob(2, jobId1, server); + checkFileCountForJob(0, jobId1, cache); + checkFileCountForJob(1, jobId2, server); + checkFileCountForJob(0, jobId2, cache); + + libCache.registerJob(jobId1, keys1, Collections.emptyList()); + ClassLoader classLoader1 = libCache.getClassLoader(jobId1); + + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId1)); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId2)); + assertEquals(2, checkFilesExist(jobId1, keys1, cache, true)); + checkFileCountForJob(2, jobId1, server); + checkFileCountForJob(2, jobId1, cache); + assertEquals(0, checkFilesExist(jobId2, keys2, cache, false)); + checkFileCountForJob(1, jobId2, server); + checkFileCountForJob(0, jobId2, cache); + + libCache.registerJob(jobId2, keys2, Collections.emptyList()); + ClassLoader classLoader2 = libCache.getClassLoader(jobId2); + assertNotEquals(classLoader1, classLoader2); - assertEquals(2, checkFilesExist(jobId, keys, server, true)); - assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries()); - assertEquals(1, libraryCacheManager.getNumberOfReferenceHolders(jid)); - - libraryCacheManager.unregisterJob(jid); - - // because we cannot guarantee that there are not thread races in the build system, we - // loop for a certain while until the references disappear - { - long deadline = System.currentTimeMillis() + 30000; - do { - Thread.sleep(500); - } - while (libraryCacheManager.getNumberOfCachedLibraries() > 0 && - System.currentTimeMillis() < deadline); + try { + libCache.registerJob(jobId2, keys1, Collections.emptyList()); + fail("Should fail with an IllegalStateException"); + } + catch (IllegalStateException e) { + // that's what we want } - - // this fails if we exited via a timeout - assertEquals(0, libraryCacheManager.getNumberOfCachedLibraries()); - assertEquals(0, libraryCacheManager.getNumberOfReferenceHolders(jid)); - - // the blob cache should no longer contain the files - assertEquals(0, checkFilesExist(jobId, keys, server, false)); try { - if (jobId == null) { - server.getFile(keys.get(0)); - } else { - server.getFile(jobId, keys.get(0)); - } - fail("BLOB should have been deleted"); - } catch (IOException e) { - // expected + libCache.registerJob( + jobId2, keys2, + Collections.singletonList(new URL("file:///tmp/does-not-exist"))); + fail("Should fail with an IllegalStateException"); } - try { - if (jobId == null) { - server.getFile(keys.get(1)); - } else { - server.getFile(jobId, keys.get(1)); - } - fail("BLOB should have been deleted"); - } catch (IOException e) { - // expected + catch (IllegalStateException e) { + // that's what we want } - } - catch (Exception e) { - e.printStackTrace(); - fail(e.getMessage()); + + assertEquals(2, libCache.getNumberOfManagedJobs()); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId1)); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId2)); + assertEquals(2, checkFilesExist(jobId1, keys1, cache, true)); + checkFileCountForJob(2, jobId1, server); + checkFileCountForJob(2, jobId1, cache); + assertEquals(1, checkFilesExist(jobId2, keys2, cache, true)); + checkFileCountForJob(1, jobId2, server); + checkFileCountForJob(1, jobId2, cache); + + libCache.unregisterJob(jobId1); + + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId1)); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId2)); + assertEquals(2, checkFilesExist(jobId1, keys1, cache, true)); + checkFileCountForJob(2, jobId1, server); + checkFileCountForJob(2, jobId1, cache); + assertEquals(1, checkFilesExist(jobId2, keys2, cache, true)); + checkFileCountForJob(1, jobId2, server); + checkFileCountForJob(1, jobId2, cache); + + libCache.unregisterJob(jobId2); + + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId1)); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId2)); + assertEquals(2, checkFilesExist(jobId1, keys1, cache, true)); + checkFileCountForJob(2, jobId1, server); + checkFileCountForJob(2, jobId1, cache); + assertEquals(1, checkFilesExist(jobId2, keys2, cache, true)); + checkFileCountForJob(1, jobId2, server); + checkFileCountForJob(1, jobId2, cache); + + // only BlobCache#releaseJob() calls clean up files (tested in BlobCacheCleanupTest etc. } finally { - if (server != null) { - server.close(); + if (libCache != null) { + libCache.shutdown(); } - if (libraryCacheManager != null) { - try { - libraryCacheManager.shutdown(); - } - catch (IOException e) { - e.printStackTrace(); - } + // should have been closed by the libraryCacheManager, but just in case + if (cache != null) { + cache.close(); + } + + if (server != null) { + server.close(); } } } /** - * Checks how many of the files given by blob keys are accessible. - * - * @param keys - * blob keys to check - * @param blobService - * BLOB store to use - * @param doThrow - * whether exceptions should be ignored (false), or throws (true) - * - * @return number of files we were able to retrieve via {@link BlobService#getFile} + * Tests that the {@link BlobLibraryCacheManager} cleans up after calling {@link + * BlobLibraryCacheManager#unregisterTask(JobID, ExecutionAttemptID)}. */ - private static int checkFilesExist( - JobID jobId, List keys, BlobService blobService, boolean doThrow) - throws IOException { - int numFiles = 0; + @Test + public void testLibraryCacheManagerTaskCleanup() throws IOException, InterruptedException { + + JobID jobId = new JobID(); + ExecutionAttemptID attempt1 = new ExecutionAttemptID(); + ExecutionAttemptID attempt2 = new ExecutionAttemptID(); + List keys = new ArrayList<>(); + BlobServer server = null; + BlobCache cache = null; + BlobLibraryCacheManager libCache = null; + + final byte[] buf = new byte[128]; + + try { + Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, + temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, 1L); + + server = new BlobServer(config, new VoidBlobStore()); + InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort()); + BlobClient bc = new BlobClient(serverAddress, config); + cache = new BlobCache(serverAddress, config, new VoidBlobStore()); + + keys.add(bc.put(jobId, buf)); + buf[0] += 1; + keys.add(bc.put(jobId, buf)); + + bc.close(); + + libCache = new BlobLibraryCacheManager(cache); + cache.registerJob(jobId); + + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(0, jobId, cache); + + libCache.registerTask(jobId, attempt1, keys, Collections.emptyList()); + ClassLoader classLoader1 = libCache.getClassLoader(jobId); + + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + libCache.registerTask(jobId, attempt2, keys, Collections.emptyList()); + ClassLoader classLoader2 = libCache.getClassLoader(jobId); + assertEquals(classLoader1, classLoader2); - for (BlobKey key : keys) { try { - if (jobId == null) { - blobService.getFile(key); - } else { - blobService.getFile(jobId, key); - } - ++numFiles; - } catch (IOException e) { - if (doThrow) { - throw e; - } + libCache.registerTask( + jobId, new ExecutionAttemptID(), Collections.emptyList(), + Collections.emptyList()); + fail("Should fail with an IllegalStateException"); + } + catch (IllegalStateException e) { + // that's what we want + } + + try { + libCache.registerTask( + jobId, new ExecutionAttemptID(), keys, + Collections.singletonList(new URL("file:///tmp/does-not-exist"))); + fail("Should fail with an IllegalStateException"); + } + catch (IllegalStateException e) { + // that's what we want } + + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(2, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + libCache.unregisterTask(jobId, attempt1); + + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + libCache.unregisterTask(jobId, attempt2); + + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + // only BlobCache#releaseJob() calls clean up files (tested in BlobCacheCleanupTest etc. } + finally { + if (libCache != null) { + libCache.shutdown(); + } + + // should have been closed by the libraryCacheManager, but just in case + if (cache != null) { + cache.close(); + } - return numFiles; + if (server != null) { + server.close(); + } + } } /** @@ -191,14 +313,14 @@ private static int checkFilesExist( * BlobLibraryCacheManager#unregisterTask(JobID, ExecutionAttemptID)}. */ @Test - public void testLibraryCacheManagerTaskCleanup() throws IOException, InterruptedException { + public void testLibraryCacheManagerMixedJobTaskCleanup() throws IOException, InterruptedException { - JobID jid = new JobID(); - ExecutionAttemptID executionId1 = new ExecutionAttemptID(); - ExecutionAttemptID executionId2 = new ExecutionAttemptID(); - List keys = new ArrayList(); + JobID jobId = new JobID(); + ExecutionAttemptID attempt1 = new ExecutionAttemptID(); + List keys = new ArrayList<>(); BlobServer server = null; - BlobLibraryCacheManager libraryCacheManager = null; + BlobCache cache = null; + BlobLibraryCacheManager libCache = null; final byte[] buf = new byte[128]; @@ -206,67 +328,96 @@ public void testLibraryCacheManagerTaskCleanup() throws IOException, Interrupted Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, 1L); server = new BlobServer(config, new VoidBlobStore()); - InetSocketAddress blobSocketAddress = new InetSocketAddress(server.getPort()); - BlobClient bc = new BlobClient(blobSocketAddress, config); - - // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager -// JobID jobId = new JobID(); - JobID jobId = null; + InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort()); + BlobClient bc = new BlobClient(serverAddress, config); + cache = new BlobCache(serverAddress, config, new VoidBlobStore()); keys.add(bc.put(jobId, buf)); buf[0] += 1; keys.add(bc.put(jobId, buf)); - long cleanupInterval = 1000L; - libraryCacheManager = new BlobLibraryCacheManager(server, cleanupInterval); - libraryCacheManager.registerTask(jid, executionId1, keys, Collections.emptyList()); - libraryCacheManager.registerTask(jid, executionId2, keys, Collections.emptyList()); + bc.close(); - assertEquals(2, checkFilesExist(jobId, keys, server, true)); - assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries()); - assertEquals(2, libraryCacheManager.getNumberOfReferenceHolders(jid)); + libCache = new BlobLibraryCacheManager(cache); + cache.registerJob(jobId); - libraryCacheManager.unregisterTask(jid, executionId1); + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(0, jobId, cache); - assertEquals(2, checkFilesExist(jobId, keys, server, true)); - assertEquals(2, libraryCacheManager.getNumberOfCachedLibraries()); - assertEquals(1, libraryCacheManager.getNumberOfReferenceHolders(jid)); + libCache.registerJob(jobId, keys, Collections.emptyList()); + ClassLoader classLoader1 = libCache.getClassLoader(jobId); - libraryCacheManager.unregisterTask(jid, executionId2); + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); - // because we cannot guarantee that there are not thread races in the build system, we - // loop for a certain while until the references disappear - { - long deadline = System.currentTimeMillis() + 30000; - do { - Thread.sleep(100); - } - while (libraryCacheManager.getNumberOfCachedLibraries() > 0 && - System.currentTimeMillis() < deadline); + libCache.registerTask(jobId, attempt1, keys, Collections.emptyList()); + ClassLoader classLoader2 = libCache.getClassLoader(jobId); + assertEquals(classLoader1, classLoader2); + + try { + libCache.registerTask( + jobId, new ExecutionAttemptID(), Collections.emptyList(), + Collections.emptyList()); + fail("Should fail with an IllegalStateException"); + } + catch (IllegalStateException e) { + // that's what we want } - // this fails if we exited via a timeout - assertEquals(0, libraryCacheManager.getNumberOfCachedLibraries()); - assertEquals(0, libraryCacheManager.getNumberOfReferenceHolders(jid)); + try { + libCache.registerTask( + jobId, new ExecutionAttemptID(), keys, + Collections.singletonList(new URL("file:///tmp/does-not-exist"))); + fail("Should fail with an IllegalStateException"); + } + catch (IllegalStateException e) { + // that's what we want + } - // the blob cache should no longer contain the files - assertEquals(0, checkFilesExist(jobId, keys, server, false)); + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(2, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); - bc.close(); - } finally { - if (server != null) { - server.close(); + libCache.unregisterJob(jobId); + + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + libCache.unregisterTask(jobId, attempt1); + + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(2, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(2, jobId, cache); + + // only BlobCache#releaseJob() calls clean up files (tested in BlobCacheCleanupTest etc. + } + finally { + if (libCache != null) { + libCache.shutdown(); } - if (libraryCacheManager != null) { - try { - libraryCacheManager.shutdown(); - } - catch (IOException e) { - e.printStackTrace(); - } + // should have been closed by the libraryCacheManager, but just in case + if (cache != null) { + cache.close(); + } + + if (server != null) { + server.close(); } } } @@ -275,75 +426,103 @@ public void testLibraryCacheManagerTaskCleanup() throws IOException, Interrupted public void testRegisterAndDownload() throws IOException { assumeTrue(!OperatingSystem.isWindows()); //setWritable doesn't work on Windows. + JobID jobId = new JobID(); BlobServer server = null; BlobCache cache = null; + BlobLibraryCacheManager libCache = null; File cacheDir = null; try { // create the blob transfer services Configuration config = new Configuration(); config.setString(BlobServerOptions.STORAGE_DIRECTORY, temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, 1_000_000L); server = new BlobServer(config, new VoidBlobStore()); InetSocketAddress serverAddress = new InetSocketAddress("localhost", server.getPort()); cache = new BlobCache(serverAddress, config, new VoidBlobStore()); - // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager - JobID jobId = null; - // upload some meaningless data to the server BlobClient uploader = new BlobClient(serverAddress, config); BlobKey dataKey1 = uploader.put(jobId, new byte[]{1, 2, 3, 4, 5, 6, 7, 8}); BlobKey dataKey2 = uploader.put(jobId, new byte[]{11, 12, 13, 14, 15, 16, 17, 18}); uploader.close(); - BlobLibraryCacheManager libCache = new BlobLibraryCacheManager(cache, 1000000000L); - - assertEquals(0, libCache.getNumberOfCachedLibraries()); + libCache = new BlobLibraryCacheManager(cache); + assertEquals(0, libCache.getNumberOfManagedJobs()); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(0, jobId, cache); // first try to access a non-existing entry + assertEquals(0, libCache.getNumberOfReferenceHolders(new JobID())); try { libCache.getClassLoader(new JobID()); fail("Should fail with an IllegalStateException"); } catch (IllegalStateException e) { - // that#s what we want + // that's what we want } - // now register some BLOBs as libraries + // register some BLOBs as libraries { - JobID jid = new JobID(); - ExecutionAttemptID executionId = new ExecutionAttemptID(); Collection keys = Collections.singleton(dataKey1); - libCache.registerTask(jid, executionId, keys, Collections.emptyList()); - assertEquals(1, libCache.getNumberOfReferenceHolders(jid)); - assertEquals(1, libCache.getNumberOfCachedLibraries()); - assertNotNull(libCache.getClassLoader(jid)); - - // un-register them again - libCache.unregisterTask(jid, executionId); + cache.registerJob(jobId); + ExecutionAttemptID executionId = new ExecutionAttemptID(); + libCache.registerTask(jobId, executionId, keys, Collections.emptyList()); + ClassLoader classLoader1 = libCache.getClassLoader(jobId); + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(1, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(1, jobId, cache); + assertNotNull(libCache.getClassLoader(jobId)); + + libCache.registerJob(jobId, keys, Collections.emptyList()); + ClassLoader classLoader2 = libCache.getClassLoader(jobId); + assertEquals(classLoader1, classLoader2); + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(2, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(1, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(1, jobId, cache); + assertNotNull(libCache.getClassLoader(jobId)); + + // un-register the job + libCache.unregisterJob(jobId); + // still one task + assertEquals(1, libCache.getNumberOfManagedJobs()); + assertEquals(1, libCache.getNumberOfReferenceHolders(jobId)); + assertEquals(1, checkFilesExist(jobId, keys, cache, true)); + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(1, jobId, cache); + + // unregister the task registration + libCache.unregisterTask(jobId, executionId); + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId)); + // changing the libCache registration does not influence the BLOB stores... + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(1, jobId, cache); // Don't fail if called again - libCache.unregisterTask(jid, executionId); + libCache.unregisterJob(jobId); + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId)); - assertEquals(0, libCache.getNumberOfReferenceHolders(jid)); + libCache.unregisterTask(jobId, executionId); + assertEquals(0, libCache.getNumberOfManagedJobs()); + assertEquals(0, libCache.getNumberOfReferenceHolders(jobId)); - // library is still cached (but not associated with job any more) - assertEquals(1, libCache.getNumberOfCachedLibraries()); + cache.releaseJob(jobId); - // should not be able to access the classloader any more - try { - libCache.getClassLoader(jid); - fail("Should fail with an IllegalStateException"); - } - catch (IllegalStateException e) { - // that's what we want - } + // library is still cached (but not associated with job any more) + checkFileCountForJob(2, jobId, server); + checkFileCountForJob(1, jobId, cache); } // see BlobUtils for the directory layout - cacheDir = new File(cache.getStorageDir(), "no_job"); + cacheDir = cache.getStorageLocation(jobId, new BlobKey()).getParentFile(); assertTrue(cacheDir.exists()); // make sure no further blobs can be downloaded by removing the write @@ -352,12 +531,14 @@ public void testRegisterAndDownload() throws IOException { // since we cannot download this library any more, this call should fail try { - libCache.registerTask(new JobID(), new ExecutionAttemptID(), Collections.singleton(dataKey2), - Collections.emptyList()); + cache.registerJob(jobId); + libCache.registerTask(jobId, new ExecutionAttemptID(), Collections.singleton(dataKey2), + Collections.emptyList()); fail("This should fail with an IOException"); } catch (IOException e) { // splendid! + cache.releaseJob(jobId); } } finally { if (cacheDir != null) { @@ -368,6 +549,9 @@ public void testRegisterAndDownload() throws IOException { if (cache != null) { cache.close(); } + if (libCache != null) { + libCache.shutdown(); + } if (server != null) { server.close(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java index 2f6738da33fd4..e52310e6b361d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.util.TestLogger; + import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -49,6 +50,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +/** + * Integration test for {@link BlobLibraryCacheManager}. + */ public class BlobLibraryCacheRecoveryITCase extends TestLogger { @Rule @@ -65,7 +69,6 @@ public void testRecoveryRegisterAndDownload() throws Exception { InetSocketAddress[] serverAddress = new InetSocketAddress[2]; BlobLibraryCacheManager[] libServer = new BlobLibraryCacheManager[2]; BlobCache cache = null; - BlobLibraryCacheManager libCache = null; BlobStoreService blobStoreService = null; Configuration config = new Configuration(); @@ -75,6 +78,7 @@ public void testRecoveryRegisterAndDownload() throws Exception { temporaryFolder.newFolder().getAbsolutePath()); config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath()); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, 3_600L); try { blobStoreService = BlobUtils.createBlobStoreFromConfig(config); @@ -82,7 +86,7 @@ public void testRecoveryRegisterAndDownload() throws Exception { for (int i = 0; i < server.length; i++) { server[i] = new BlobServer(config, blobStoreService); serverAddress[i] = new InetSocketAddress("localhost", server[i].getPort()); - libServer[i] = new BlobLibraryCacheManager(server[i], 3600 * 1000); + libServer[i] = new BlobLibraryCacheManager(server[i]); } // Random data @@ -92,25 +96,22 @@ public void testRecoveryRegisterAndDownload() throws Exception { List keys = new ArrayList<>(2); JobID jobId = new JobID(); - // TODO: replace+adapt by jobId after adapting the BlobLibraryCacheManager - JobID blobJobId = null; // Upload some data (libraries) try (BlobClient client = new BlobClient(serverAddress[0], config)) { - keys.add(client.put(blobJobId, expected)); // Request 1 - keys.add(client.put(blobJobId, expected, 32, 256)); // Request 2 + keys.add(client.put(jobId, expected)); // Request 1 + keys.add(client.put(jobId, expected, 32, 256)); // Request 2 } // The cache cache = new BlobCache(serverAddress[0], config, blobStoreService); - libCache = new BlobLibraryCacheManager(cache, 3600 * 1000); // Register uploaded libraries ExecutionAttemptID executionId = new ExecutionAttemptID(); libServer[0].registerTask(jobId, executionId, keys, Collections.emptyList()); // Verify key 1 - File f = cache.getFile(keys.get(0)); + File f = cache.getFile(jobId, keys.get(0)); assertEquals(expected.length, f.length()); try (FileInputStream fis = new FileInputStream(f)) { @@ -123,13 +124,11 @@ public void testRecoveryRegisterAndDownload() throws Exception { // Shutdown cache and start with other server cache.close(); - libCache.shutdown(); cache = new BlobCache(serverAddress[1], config, blobStoreService); - libCache = new BlobLibraryCacheManager(cache, 3600 * 1000); // Verify key 1 - f = cache.getFile(keys.get(0)); + f = cache.getFile(jobId, keys.get(0)); assertEquals(expected.length, f.length()); try (FileInputStream fis = new FileInputStream(f)) { @@ -141,7 +140,7 @@ public void testRecoveryRegisterAndDownload() throws Exception { } // Verify key 2 - f = cache.getFile(keys.get(1)); + f = cache.getFile(jobId, keys.get(1)); assertEquals(256, f.length()); try (FileInputStream fis = new FileInputStream(f)) { @@ -154,8 +153,8 @@ public void testRecoveryRegisterAndDownload() throws Exception { // Remove blobs again try (BlobClient client = new BlobClient(serverAddress[1], config)) { - client.delete(keys.get(0)); - client.delete(keys.get(1)); + client.delete(jobId, keys.get(0)); + client.delete(jobId, keys.get(1)); } // Verify everything is clean below recoveryDir/ @@ -167,6 +166,11 @@ public void testRecoveryRegisterAndDownload() throws Exception { assertEquals("Unclean state backend: " + Arrays.toString(recoveryFiles), 0, recoveryFiles.length); } finally { + for (BlobLibraryCacheManager s : libServer) { + if (s != null) { + s.shutdown(); + } + } for (BlobServer s : server) { if (s != null) { s.close(); @@ -177,10 +181,6 @@ public void testRecoveryRegisterAndDownload() throws Exception { cache.close(); } - if (libCache != null) { - libCache.shutdown(); - } - if (blobStoreService != null) { blobStoreService.closeAndCleanupAllData(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerCleanupITCase.java new file mode 100644 index 0000000000000..b2b455b8d7659 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerCleanupITCase.java @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmanager; + +import akka.actor.ActorSystem; +import akka.testkit.JavaTestKit; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.ConfigConstants; +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.blob.BlobClient; +import org.apache.flink.runtime.blob.BlobKey; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +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.jobgraph.JobVertex; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.testingUtils.TestingCluster; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.runtime.testtasks.FailingBlockingInvokable; +import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import scala.concurrent.Await; +import scala.concurrent.Future; +import scala.concurrent.duration.FiniteDuration; + +import java.io.File; +import java.io.FilenameFilter; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Arrays; + +import static org.apache.flink.runtime.testingUtils.TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; + +/** + * Small test to check that the {@link org.apache.flink.runtime.blob.BlobServer} cleanup is executed + * after job termination. + */ +public class JobManagerCleanupITCase extends TestLogger { + + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + + private static ActorSystem system; + + @BeforeClass + public static void setup() { + system = AkkaUtils.createLocalActorSystem(new Configuration()); + } + + @AfterClass + public static void teardown() { + JavaTestKit.shutdownActorSystem(system); + } + + /** + * Specifies which test case to run in {@link #testBlobServerCleanup(TestCase)}. + */ + private enum TestCase { + JOB_FINISHES_SUCESSFULLY, + JOB_IS_CANCELLED, + JOB_FAILS, + JOB_SUBMISSION_FAILS + } + + /** + * Test cleanup for a job that finishes ordinarily. + */ + @Test + public void testBlobServerCleanupFinishedJob() throws IOException { + testBlobServerCleanup(TestCase.JOB_FINISHES_SUCESSFULLY); + } + + /** + * Test cleanup for a job which is cancelled after submission. + */ + @Test + public void testBlobServerCleanupCancelledJob() throws IOException { + testBlobServerCleanup(TestCase.JOB_IS_CANCELLED); + } + + /** + * Test cleanup for a job that fails (first a task fails, then the job recovers, then the whole + * job fails due to a limited restart policy). + */ + @Test + public void testBlobServerCleanupFailedJob() throws IOException { + testBlobServerCleanup(TestCase.JOB_FAILS); + } + + /** + * Test cleanup for a job that fails job submission (emulated by an additional BLOB not being + * present). + */ + @Test + public void testBlobServerCleanupFailedSubmission() throws IOException { + testBlobServerCleanup(TestCase.JOB_SUBMISSION_FAILS); + } + + private void testBlobServerCleanup(final TestCase testCase) throws IOException { + final int num_tasks = 2; + final File blobBaseDir = tmpFolder.newFolder(); + + new JavaTestKit(system) {{ + new Within(duration("30 seconds")) { + @Override + protected void run() { + // Setup + + TestingCluster cluster = null; + BlobClient bc = null; + + try { + Configuration config = new Configuration(); + config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2); + config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 1); + config.setString(AkkaOptions.ASK_TIMEOUT, DEFAULT_AKKA_ASK_TIMEOUT()); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, blobBaseDir.getAbsolutePath()); + + config.setString(ConfigConstants.RESTART_STRATEGY, "fixeddelay"); + config.setInteger(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1); + config.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "1 s"); + // BLOBs are deleted from BlobCache between 1s and 2s after last reference + // -> the BlobCache may still have the BLOB or not (let's test both cases randomly) + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, 1L); + + cluster = new TestingCluster(config); + cluster.start(); + + final ActorGateway jobManagerGateway = cluster.getLeaderGateway( + TestingUtils.TESTING_DURATION()); + + // we can set the leader session ID to None because we don't use this gateway to send messages + final ActorGateway testActorGateway = new AkkaActorGateway(getTestActor(), + HighAvailabilityServices.DEFAULT_LEADER_ID); + + // Create a task + + JobVertex source = new JobVertex("Source"); + if (testCase == TestCase.JOB_FAILS || testCase == TestCase.JOB_IS_CANCELLED) { + source.setInvokableClass(FailingBlockingInvokable.class); + } else { + source.setInvokableClass(NoOpInvokable.class); + } + source.setParallelism(num_tasks); + + JobGraph jobGraph = new JobGraph("BlobCleanupTest", source); + final JobID jid = jobGraph.getJobID(); + + // request the blob port from the job manager + Future future = jobManagerGateway + .ask(JobManagerMessages.getRequestBlobManagerPort(), remaining()); + int blobPort = (Integer) Await.result(future, remaining()); + + // upload a blob + BlobKey key1; + bc = new BlobClient(new InetSocketAddress("localhost", blobPort), + config); + try { + key1 = bc.put(jid, new byte[10]); + } finally { + bc.close(); + } + jobGraph.addBlob(key1); + + if (testCase == TestCase.JOB_SUBMISSION_FAILS) { + // add an invalid key so that the submission fails + jobGraph.addBlob(new BlobKey()); + } + + // Submit the job and wait for all vertices to be running + jobManagerGateway.tell( + new JobManagerMessages.SubmitJob( + jobGraph, + ListeningBehaviour.EXECUTION_RESULT), + testActorGateway); + if (testCase == TestCase.JOB_SUBMISSION_FAILS) { + expectMsgClass(JobManagerMessages.JobResultFailure.class); + } else { + expectMsgClass(JobManagerMessages.JobSubmitSuccess.class); + + if (testCase == TestCase.JOB_FAILS) { + // fail a task so that the job is going to be recovered (we actually do not + // need the blocking part of the invokable and can start throwing right away) + FailingBlockingInvokable.unblock(); + + // job will get restarted, BlobCache may re-download the BLOB if already deleted + // then the tasks will fail again and the restart strategy will finalise the job + + expectMsgClass(JobManagerMessages.JobResultFailure.class); + } else if (testCase == TestCase.JOB_IS_CANCELLED) { + jobManagerGateway.tell( + new JobManagerMessages.CancelJob(jid), + testActorGateway); + expectMsgClass(JobManagerMessages.CancellationResponse.class); + + // job will be cancelled and everything should be cleaned up + + expectMsgClass(JobManagerMessages.JobResultFailure.class); + } else { + expectMsgClass(JobManagerMessages.JobResultSuccess.class); + } + } + + // both BlobServer and BlobCache should eventually delete all files + + File[] blobDirs = blobBaseDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.startsWith("blobStore-"); + } + }); + assertNotNull(blobDirs); + for (File blobDir : blobDirs) { + waitForEmptyBlobDir(blobDir, remaining()); + } + + } catch (Exception e) { + e.printStackTrace(); + fail(e.getMessage()); + } finally { + if (bc != null) { + try { + bc.close(); + } catch (IOException ignored) { + } + } + if (cluster != null) { + cluster.shutdown(); + } + } + } + }; + }}; + + // after everything has been shut down, the storage directory itself should be empty + assertArrayEquals(new File[] {}, blobBaseDir.listFiles()); + } + + /** + * Waits until the given {@link org.apache.flink.runtime.blob.BlobService} storage directory + * does not contain any job-related folders any more. + * + * @param blobDir + * directory of a {@link org.apache.flink.runtime.blob.BlobServer} or {@link + * org.apache.flink.runtime.blob.BlobCache} + * @param remaining + * remaining time for this test + * + * @see org.apache.flink.runtime.blob.BlobUtils + */ + private static void waitForEmptyBlobDir(File blobDir, FiniteDuration remaining) + throws InterruptedException { + long deadline = System.currentTimeMillis() + remaining.toMillis(); + String[] blobDirContents; + do { + blobDirContents = blobDir.list(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.startsWith("job_"); + } + }); + if (blobDirContents == null || blobDirContents.length == 0) { + return; + } + Thread.sleep(100); + } while (System.currentTimeMillis() < deadline); + + fail("Timeout while waiting for " + blobDir.getAbsolutePath() + " to become empty. Current contents: " + Arrays.toString(blobDirContents)); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java index 23f0a389076c3..933c7a074ff04 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerHARecoveryTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; @@ -27,7 +28,6 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.akka.ListeningBehaviour; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.BlobService; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; @@ -90,6 +90,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; + import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -158,6 +159,7 @@ public void testJobRecoveryWhenLosingLeadership() throws Exception { flinkConfiguration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); flinkConfiguration.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().toString()); flinkConfiguration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, slots); + flinkConfiguration.setLong(BlobServerOptions.CLEANUP_INTERVAL, 3_600L); try { Scheduler scheduler = new Scheduler(TestingUtils.defaultExecutionContext()); @@ -179,6 +181,9 @@ public void testJobRecoveryWhenLosingLeadership() throws Exception { archive = system.actorOf(JobManager.getArchiveProps(MemoryArchivist.class, 10, Option.empty())); + BlobServer blobServer = new BlobServer( + flinkConfiguration, + testingHighAvailabilityServices.createBlobStore()); Props jobManagerProps = Props.create( TestingJobManager.class, flinkConfiguration, @@ -186,11 +191,8 @@ public void testJobRecoveryWhenLosingLeadership() throws Exception { TestingUtils.defaultExecutor(), instanceManager, scheduler, - new BlobLibraryCacheManager( - new BlobServer( - flinkConfiguration, - testingHighAvailabilityServices.createBlobStore()), - 3600000L), + blobServer, + new BlobLibraryCacheManager(blobServer), archive, new FixedDelayRestartStrategy.FixedDelayRestartStrategyFactory(Int.MaxValue(), 100), timeout, @@ -353,6 +355,7 @@ public void testFailingJobRecovery() throws Exception { final Collection recoveredJobs = new ArrayList<>(2); + BlobServer blobServer = mock(BlobServer.class); Props jobManagerProps = Props.create( TestingFailingHAJobManager.class, flinkConfiguration, @@ -360,7 +363,8 @@ public void testFailingJobRecovery() throws Exception { TestingUtils.defaultExecutor(), mock(InstanceManager.class), mock(Scheduler.class), - new BlobLibraryCacheManager(mock(BlobService.class), 1 << 20), + blobServer, + new BlobLibraryCacheManager(blobServer), ActorRef.noSender(), new FixedDelayRestartStrategy.FixedDelayRestartStrategyFactory(Int.MaxValue(), 100), timeout, @@ -397,6 +401,7 @@ public TestingFailingHAJobManager( Executor ioExecutor, InstanceManager instanceManager, Scheduler scheduler, + BlobServer blobServer, BlobLibraryCacheManager libraryCacheManager, ActorRef archive, RestartStrategyFactory restartStrategyFactory, @@ -413,6 +418,7 @@ public TestingFailingHAJobManager( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java index 3c75971f5c7c8..6a39293342d92 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobSubmitTest.java @@ -137,14 +137,13 @@ public void testFailureWhenJarBlobsMissing() { // upload two dummy bytes and add their keys to the job graph as dependencies BlobKey key1, key2; BlobClient bc = new BlobClient(new InetSocketAddress("localhost", blobPort), jmConfig); - // TODO: make use of job-related BLOBs after adapting the BlobLibraryCacheManager - JobID jobId = null; + JobID jobId = jg.getJobID(); try { key1 = bc.put(jobId, new byte[10]); key2 = bc.put(jobId, new byte[10]); // delete one of the blobs to make sure that the startup failed - bc.delete(key2); + bc.delete(jobId, key2); } finally { bc.close(); 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 df353690761ca..2c17b5a78587e 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.ScheduledExecutor; @@ -92,8 +93,8 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { final ScheduledExecutor scheduledExecutor = mock(ScheduledExecutor.class); final HeartbeatServices heartbeatServices = new TestingHeartbeatServices(heartbeatInterval, heartbeatTimeout, scheduledExecutor); - final BlobLibraryCacheManager libraryCacheManager = mock(BlobLibraryCacheManager.class); - when(libraryCacheManager.getBlobServerPort()).thenReturn(1337); + BlobServer blobServer = mock(BlobServer.class); + when(blobServer.getPort()).thenReturn(1337); final JobGraph jobGraph = new JobGraph(); @@ -106,7 +107,8 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { haServices, heartbeatServices, Executors.newScheduledThreadPool(1), - libraryCacheManager, + blobServer, + mock(BlobLibraryCacheManager.class), mock(RestartStrategyFactory.class), Time.of(10, TimeUnit.SECONDS), null, @@ -204,6 +206,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { haServices, heartbeatServices, Executors.newScheduledThreadPool(1), + mock(BlobServer.class), mock(BlobLibraryCacheManager.class), mock(RestartStrategyFactory.class), Time.of(10, TimeUnit.SECONDS), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java index 70800e50ad9d7..230ca91417792 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/JobManagerLeaderElectionTest.java @@ -25,9 +25,10 @@ import akka.pattern.Patterns; import akka.testkit.JavaTestKit; import akka.util.Timeout; - import org.apache.curator.framework.CuratorFramework; import org.apache.curator.test.TestingServer; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.blob.BlobServer; @@ -47,13 +48,11 @@ import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.util.TestLogger; - import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; - import scala.Option; import scala.concurrent.Await; import scala.concurrent.Future; @@ -178,6 +177,9 @@ private Props createJobManagerProps(Configuration configuration) throws Exceptio SubmittedJobGraphStore submittedJobGraphStore = new StandaloneSubmittedJobGraphStore(); CheckpointRecoveryFactory checkpointRecoveryFactory = new StandaloneCheckpointRecoveryFactory(); + configuration.setLong(BlobServerOptions.CLEANUP_INTERVAL, 1L); + + BlobServer blobServer = new BlobServer(configuration, new VoidBlobStore()); return Props.create( TestingJobManager.class, configuration, @@ -185,7 +187,8 @@ private Props createJobManagerProps(Configuration configuration) throws Exceptio TestingUtils.defaultExecutor(), new InstanceManager(), new Scheduler(TestingUtils.defaultExecutionContext()), - new BlobLibraryCacheManager(new BlobServer(configuration, new VoidBlobStore()), 10L), + blobServer, + new BlobLibraryCacheManager(blobServer), ActorRef.noSender(), new NoRestartStrategy.NoRestartStrategyFactory(), AkkaUtils.getDefaultTimeoutAsFiniteDuration(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 43ff60bd90d0e..6842bee4a4057 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -668,7 +669,7 @@ public void testTaskSubmission() throws Exception { Collections.emptyList()); final LibraryCacheManager libraryCacheManager = mock(LibraryCacheManager.class); - when(libraryCacheManager.getClassLoader(eq(jobId))).thenReturn(getClass().getClassLoader()); + when(libraryCacheManager.getClassLoader(any(JobID.class))).thenReturn(ClassLoader.getSystemClassLoader()); final JobManagerConnection jobManagerConnection = new JobManagerConnection( jobId, @@ -677,6 +678,7 @@ public void testTaskSubmission() throws Exception { jobManagerLeaderId, mock(TaskManagerActions.class), mock(CheckpointResponder.class), + mock(BlobCache.class), libraryCacheManager, mock(ResultPartitionConsumableNotifier.class), mock(PartitionProducerStateChecker.class)); @@ -1191,6 +1193,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { jobManagerLeaderId, mock(TaskManagerActions.class), mock(CheckpointResponder.class), + mock(BlobCache.class), libraryCacheManager, mock(ResultPartitionConsumableNotifier.class), mock(PartitionProducerStateChecker.class)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 085a38699772f..392dc29bf3ca8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; @@ -145,6 +146,7 @@ public void testMixedAsyncCallsInOrder() { } private static Task createTask() throws Exception { + BlobCache blobCache = mock(BlobCache.class); LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(ClassLoader.getSystemClassLoader()); @@ -195,6 +197,7 @@ private static Task createTask() throws Exception { mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), + blobCache, libCache, mock(FileCache.class), new TestingTaskManagerRuntimeInfo(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java index 1ebd4adf73883..ac0df3663f4dc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -98,6 +99,7 @@ public void doMocking(AbstractInvokable taskMock) throws Exception { mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), + mock(BlobCache.class), mock(LibraryCacheManager.class), mock(FileCache.class), tmRuntimeInfo, 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 ba3e8201f0ce2..d4cd0cfcf64d5 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 @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -227,7 +228,8 @@ public void testFailExternallyRightAway() { @Test public void testLibraryCacheRegistrationFailed() { try { - Task task = createTask(TestInvokableCorrect.class, mock(LibraryCacheManager.class)); + Task task = createTask(TestInvokableCorrect.class, mock(BlobCache.class), + mock(LibraryCacheManager.class)); // task should be new and perfect assertEquals(ExecutionState.CREATED, task.getExecutionState()); @@ -260,6 +262,7 @@ public void testLibraryCacheRegistrationFailed() { @Test public void testExecutionFailsInNetworkRegistration() { try { + BlobCache blobCache = mock(BlobCache.class); // mock a working library cache LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(getClass().getClassLoader()); @@ -274,7 +277,7 @@ public void testExecutionFailsInNetworkRegistration() { when(network.getDefaultIOMode()).thenReturn(IOManager.IOMode.SYNC); doThrow(new RuntimeException("buffers")).when(network).registerTask(any(Task.class)); - Task task = createTask(TestInvokableCorrect.class, libCache, network, consumableNotifier, partitionProducerStateChecker, executor); + Task task = createTask(TestInvokableCorrect.class, blobCache, libCache, network, consumableNotifier, partitionProducerStateChecker, executor); task.registerExecutionListener(listener); @@ -617,6 +620,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { IntermediateDataSetID resultId = new IntermediateDataSetID(); ResultPartitionID partitionId = new ResultPartitionID(); + BlobCache blobCache = mock(BlobCache.class); LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(getClass().getClassLoader()); @@ -629,7 +633,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { when(network.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class))) .thenReturn(mock(TaskKvStateRegistry.class)); - createTask(InvokableBlockingInInvoke.class, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); + createTask(InvokableBlockingInInvoke.class, blobCache, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); // Test all branches of trigger partition state check @@ -638,7 +642,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { createQueuesAndActors(); // PartitionProducerDisposedException - Task task = createTask(InvokableBlockingInInvoke.class, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); + Task task = createTask(InvokableBlockingInInvoke.class, blobCache, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); CompletableFuture promise = new CompletableFuture<>(); when(partitionChecker.requestPartitionProducerState(eq(task.getJobID()), eq(resultId), eq(partitionId))).thenReturn(promise); @@ -654,7 +658,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { createQueuesAndActors(); // Any other exception - Task task = createTask(InvokableBlockingInInvoke.class, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); + Task task = createTask(InvokableBlockingInInvoke.class, blobCache, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); CompletableFuture promise = new CompletableFuture<>(); when(partitionChecker.requestPartitionProducerState(eq(task.getJobID()), eq(resultId), eq(partitionId))).thenReturn(promise); @@ -671,7 +675,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { createQueuesAndActors(); // TimeoutException handled special => retry - Task task = createTask(InvokableBlockingInInvoke.class, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); + Task task = createTask(InvokableBlockingInInvoke.class, blobCache, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); SingleInputGate inputGate = mock(SingleInputGate.class); when(inputGate.getConsumedResultId()).thenReturn(resultId); @@ -702,7 +706,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { createQueuesAndActors(); // Success - Task task = createTask(InvokableBlockingInInvoke.class, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); + Task task = createTask(InvokableBlockingInInvoke.class, blobCache, libCache, network, consumableNotifier, partitionChecker, Executors.directExecutor()); SingleInputGate inputGate = mock(SingleInputGate.class); when(inputGate.getConsumedResultId()).thenReturn(resultId); @@ -882,26 +886,30 @@ private Task createTask(Class invokable) throws IOE } private Task createTask(Class invokable, Configuration config) throws IOException { + BlobCache blobCache = mock(BlobCache.class); LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(getClass().getClassLoader()); - return createTask(invokable, libCache, config, new ExecutionConfig()); + return createTask(invokable, blobCache,libCache, config, new ExecutionConfig()); } private Task createTask(Class invokable, Configuration config, ExecutionConfig execConfig) throws IOException { + BlobCache blobCache = mock(BlobCache.class); LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(getClass().getClassLoader()); - return createTask(invokable, libCache, config, execConfig); + return createTask(invokable, blobCache,libCache, config, execConfig); } private Task createTask( Class invokable, + BlobCache blobCache, LibraryCacheManager libCache) throws IOException { - return createTask(invokable, libCache, new Configuration(), new ExecutionConfig()); + return createTask(invokable, blobCache,libCache, new Configuration(), new ExecutionConfig()); } private Task createTask( Class invokable, + BlobCache blobCache, LibraryCacheManager libCache, Configuration config, ExecutionConfig execConfig) throws IOException { @@ -916,21 +924,23 @@ private Task createTask( when(network.createKvStateTaskRegistry(any(JobID.class), any(JobVertexID.class))) .thenReturn(mock(TaskKvStateRegistry.class)); - return createTask(invokable, libCache, network, consumableNotifier, partitionProducerStateChecker, executor, config, execConfig); + return createTask(invokable, blobCache, libCache, network, consumableNotifier, partitionProducerStateChecker, executor, config, execConfig); } private Task createTask( Class invokable, + BlobCache blobCache, LibraryCacheManager libCache, NetworkEnvironment networkEnvironment, ResultPartitionConsumableNotifier consumableNotifier, PartitionProducerStateChecker partitionProducerStateChecker, Executor executor) throws IOException { - return createTask(invokable, libCache, networkEnvironment, consumableNotifier, partitionProducerStateChecker, executor, new Configuration(), new ExecutionConfig()); + return createTask(invokable, blobCache, libCache, networkEnvironment, consumableNotifier, partitionProducerStateChecker, executor, new Configuration(), new ExecutionConfig()); } private Task createTask( Class invokable, + BlobCache blobCache, LibraryCacheManager libCache, NetworkEnvironment networkEnvironment, ResultPartitionConsumableNotifier consumableNotifier, @@ -991,6 +1001,7 @@ private Task createTask( taskManagerConnection, inputSplitProvider, checkpointResponder, + blobCache, libCache, mock(FileCache.class), new TestingTaskManagerRuntimeInfo(taskManagerConfig), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/FailingBlockingInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/FailingBlockingInvokable.java new file mode 100644 index 0000000000000..37c141d6b9f02 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testtasks/FailingBlockingInvokable.java @@ -0,0 +1,48 @@ +/* + * 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.testtasks; + +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; + +/** + * Task which blocks until the (static) {@link #unblock()} method is called and then fails with an + * exception. + */ +public class FailingBlockingInvokable extends AbstractInvokable { + private static volatile boolean blocking = true; + private static final Object lock = new Object(); + + @Override + public void invoke() throws Exception { + while (blocking) { + synchronized (lock) { + lock.wait(); + } + } + throw new RuntimeException("This exception is expected."); + } + + public static void unblock() { + blocking = false; + + synchronized (lock) { + lock.notifyAll(); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java index c1df5a376d492..229f1eb08b859 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; @@ -178,6 +179,7 @@ public static void main(String[] args) throws Exception { new NoOpTaskManagerActions(), new NoOpInputSplitProvider(), new NoOpCheckpointResponder(), + mock(BlobCache.class), new FallbackLibraryCacheManager(), new FileCache(tmInfo.getTmpDirectories()), tmInfo, diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala index 1b9ee48646c73..95da9814bdbaa 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/jobmanager/JobManagerRegistrationTest.scala @@ -264,14 +264,15 @@ ImplicitSender with WordSpecLike with Matchers with BeforeAndAfterAll with Befor components._1, components._2, components._3, - ActorRef.noSender, components._4, + ActorRef.noSender, components._5, + components._6, highAvailabilityServices.getJobManagerLeaderElectionService( HighAvailabilityServices.DEFAULT_JOB_ID), highAvailabilityServices.getSubmittedJobGraphStore(), highAvailabilityServices.getCheckpointRecoveryFactory(), - components._8, + components._9, None) _system.actorOf(props) diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala index e5655bb5f5ab4..87f80882730d1 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala @@ -28,6 +28,7 @@ import akka.testkit.CallingThreadDispatcher import org.apache.flink.api.common.JobID import org.apache.flink.configuration.{Configuration, JobManagerOptions} import org.apache.flink.runtime.akka.AkkaUtils +import org.apache.flink.runtime.blob.BlobServer import org.apache.flink.runtime.checkpoint.savepoint.Savepoint import org.apache.flink.runtime.checkpoint.{CheckpointOptions, CheckpointRecoveryFactory} import org.apache.flink.runtime.clusterframework.FlinkResourceManager @@ -110,6 +111,7 @@ class TestingCluster( ioExecutor: Executor, instanceManager: InstanceManager, scheduler: Scheduler, + blobServer: BlobServer, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, restartStrategyFactory: RestartStrategyFactory, @@ -127,6 +129,7 @@ class TestingCluster( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala index f50a832f6b92a..8b9ce15b142a6 100644 --- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala +++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{Executor, ScheduledExecutorService} import akka.actor.ActorRef import org.apache.flink.configuration.Configuration +import org.apache.flink.runtime.blob.BlobServer import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory @@ -34,15 +35,16 @@ import org.apache.flink.runtime.metrics.MetricRegistry import scala.concurrent.duration._ import scala.language.postfixOps -/** JobManager implementation extended by testing messages - * - */ +/** + * JobManager implementation extended by testing messages + */ class TestingJobManager( flinkConfiguration: Configuration, futureExecutor: ScheduledExecutorService, ioExecutor: Executor, instanceManager: InstanceManager, scheduler: Scheduler, + blobServer: BlobServer, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, restartStrategyFactory: RestartStrategyFactory, @@ -58,6 +60,7 @@ class TestingJobManager( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java index 3b8178bb16fbf..82642eab4dcc7 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/BlockingCheckpointsTest.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; @@ -156,6 +157,7 @@ private static Task createTask(Configuration taskConfig) throws IOException { mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), + mock(BlobCache.class), new FallbackLibraryCacheManager(), new FileCache(new String[] { EnvironmentInformation.getTemporaryFileDirectory() }), new TestingTaskManagerRuntimeInfo(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index 82e4f3174fb42..14ae733164133 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -274,6 +275,7 @@ private static Task createTask( mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), + mock(BlobCache.class), new FallbackLibraryCacheManager(), new FileCache(new String[] { EnvironmentInformation.getTemporaryFileDirectory() }), new TestingTaskManagerRuntimeInfo(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 702d833ad19ae..79e9583a8be72 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointOptions; @@ -153,6 +154,7 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), + mock(BlobCache.class), new FallbackLibraryCacheManager(), mock(FileCache.class), taskManagerRuntimeInfo, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 09e9a1b26144c..08c3207b546e5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -27,6 +27,7 @@ import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.blob.BlobCache; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; @@ -796,6 +797,7 @@ public static Task createTask( StreamConfig taskConfig, Configuration taskManagerConfig) throws Exception { + BlobCache blobCache = mock(BlobCache.class); LibraryCacheManager libCache = mock(LibraryCacheManager.class); when(libCache.getClassLoader(any(JobID.class))).thenReturn(StreamTaskTest.class.getClassLoader()); @@ -844,6 +846,7 @@ public static Task createTask( mock(TaskManagerActions.class), mock(InputSplitProvider.class), mock(CheckpointResponder.class), + blobCache, libCache, mock(FileCache.class), new TestingTaskManagerRuntimeInfo(taskManagerConfig, new String[] {System.getProperty("java.io.tmpdir")}), diff --git a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnJobManager.scala b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnJobManager.scala index b539961d8b638..bd72d6d5b721e 100644 --- a/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnJobManager.scala +++ b/flink-yarn-tests/src/test/scala/org/apache/flink/yarn/TestingYarnJobManager.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{Executor, ScheduledExecutorService} import akka.actor.ActorRef import org.apache.flink.configuration.Configuration +import org.apache.flink.runtime.blob.BlobServer import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory @@ -58,6 +59,7 @@ class TestingYarnJobManager( ioExecutor: Executor, instanceManager: InstanceManager, scheduler: Scheduler, + blobServer: BlobServer, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, restartStrategyFactory: RestartStrategyFactory, @@ -73,6 +75,7 @@ class TestingYarnJobManager( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala index a2d166854fb1c..b8dacee3b90b2 100644 --- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala +++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala @@ -24,6 +24,7 @@ import java.util.concurrent.{Executor, ScheduledExecutorService, TimeUnit} import akka.actor.ActorRef import org.apache.flink.configuration.{Configuration => FlinkConfiguration} import org.apache.flink.core.fs.Path +import org.apache.flink.runtime.blob.BlobServer import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory import org.apache.flink.runtime.clusterframework.ContaineredJobManager import org.apache.flink.runtime.clusterframework.messages.StopCluster @@ -49,7 +50,8 @@ import scala.language.postfixOps * @param instanceManager Instance manager to manage the registered * [[org.apache.flink.runtime.taskmanager.TaskManager]] * @param scheduler Scheduler to schedule Flink jobs - * @param libraryCacheManager Manager to manage uploaded jar files + * @param blobServer BLOB store for file uploads + * @param libraryCacheManager manages uploaded jar files and class paths * @param archive Archive for finished Flink jobs * @param restartStrategyFactory Restart strategy to be used in case of a job recovery * @param timeout Timeout for futures @@ -61,6 +63,7 @@ class YarnJobManager( ioExecutor: Executor, instanceManager: InstanceManager, scheduler: FlinkScheduler, + blobServer: BlobServer, libraryCacheManager: BlobLibraryCacheManager, archive: ActorRef, restartStrategyFactory: RestartStrategyFactory, @@ -76,6 +79,7 @@ class YarnJobManager( ioExecutor, instanceManager, scheduler, + blobServer, libraryCacheManager, archive, restartStrategyFactory, From 76f1022884fe7b291fe81028a29896fb5b5ca5c9 Mon Sep 17 00:00:00 2001 From: Yonatan Most Date: Wed, 2 Aug 2017 17:58:50 +0300 Subject: [PATCH 028/129] [FLINK-7347] Keep IDs for checkpoint in a set in MessageQueue Source Previously, they were kept in a List, which made removal from the Set of unconfirmed IDs prohibitively expensive in MessageAcknowledgingSourceBase.notifyCheckpointComplete(). --- .../connectors/rabbitmq/RMQSourceTest.java | 12 +++++----- .../state/SerializedCheckpointData.java | 23 +++++++++---------- .../MessageAcknowledgingSourceBase.java | 16 ++++++------- ...ipleIdsMessageAcknowledgingSourceBase.java | 4 +++- 4 files changed, 28 insertions(+), 27 deletions(-) diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java index 05ae8108b5a68..f180e786c2f28 100644 --- a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java +++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceTest.java @@ -51,8 +51,8 @@ import java.io.IOException; import java.util.ArrayDeque; -import java.util.List; import java.util.Random; +import java.util.Set; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -180,12 +180,12 @@ public void testCheckpointing() throws Exception { testHarnessCopy.initializeState(data); testHarnessCopy.open(); - ArrayDeque>> deque = sourceCopy.getRestoredState(); - List messageIds = deque.getLast().f1; + ArrayDeque>> deque = sourceCopy.getRestoredState(); + Set messageIds = deque.getLast().f1; assertEquals(numIds, messageIds.size()); if (messageIds.size() > 0) { - assertEquals(lastSnapshotId, (long) Long.valueOf(messageIds.get(messageIds.size() - 1))); + assertTrue(messageIds.contains(Long.toString(lastSnapshotId))); } // check if the messages are being acknowledged and the transaction committed @@ -339,7 +339,7 @@ public TypeInformation getProducedType() { private class RMQTestSource extends RMQSource { - private ArrayDeque>> restoredState; + private ArrayDeque>> restoredState; public RMQTestSource() { super(new RMQConnectionConfig.Builder().setHost("hostTest") @@ -353,7 +353,7 @@ public void initializeState(FunctionInitializationContext context) throws Except this.restoredState = this.pendingCheckpoints; } - public ArrayDeque>> getRestoredState() { + public ArrayDeque>> getRestoredState() { return this.restoredState; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SerializedCheckpointData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SerializedCheckpointData.java index 16ad3fde602b1..394791b9014e2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SerializedCheckpointData.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SerializedCheckpointData.java @@ -25,8 +25,8 @@ import java.io.IOException; import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.List; +import java.util.HashSet; +import java.util.Set; /** * This class represents serialized checkpoint data for a collection of elements. @@ -95,7 +95,7 @@ public int getNumIds() { * * @throws IOException Thrown, if the serialization fails. */ - public static SerializedCheckpointData[] fromDeque(ArrayDeque>> checkpoints, + public static SerializedCheckpointData[] fromDeque(ArrayDeque>> checkpoints, TypeSerializer serializer) throws IOException { return fromDeque(checkpoints, serializer, new DataOutputSerializer(128)); } @@ -111,15 +111,15 @@ public static SerializedCheckpointData[] fromDeque(ArrayDeque SerializedCheckpointData[] fromDeque(ArrayDeque>> checkpoints, + public static SerializedCheckpointData[] fromDeque(ArrayDeque>> checkpoints, TypeSerializer serializer, DataOutputSerializer outputBuffer) throws IOException { SerializedCheckpointData[] serializedCheckpoints = new SerializedCheckpointData[checkpoints.size()]; int pos = 0; - for (Tuple2> checkpoint : checkpoints) { + for (Tuple2> checkpoint : checkpoints) { outputBuffer.clear(); - List checkpointIds = checkpoint.f1; + Set checkpointIds = checkpoint.f1; for (T id : checkpointIds) { serializer.serialize(id, outputBuffer); @@ -146,10 +146,9 @@ public static SerializedCheckpointData[] fromDeque(ArrayDeque ArrayDeque>> toDeque( - SerializedCheckpointData[] data, TypeSerializer serializer) throws IOException - { - ArrayDeque>> deque = new ArrayDeque<>(data.length); + public static ArrayDeque>> toDeque( + SerializedCheckpointData[] data, TypeSerializer serializer) throws IOException { + ArrayDeque>> deque = new ArrayDeque<>(data.length); DataInputDeserializer deser = null; for (SerializedCheckpointData checkpoint : data) { @@ -161,14 +160,14 @@ public static ArrayDeque>> toDeque( deser.setBuffer(serializedData, 0, serializedData.length); } - final List ids = new ArrayList<>(checkpoint.getNumIds()); + final Set ids = new HashSet<>(checkpoint.getNumIds()); final int numIds = checkpoint.getNumIds(); for (int i = 0; i < numIds; i++) { ids.add(serializer.deserialize(deser)); } - deque.addLast(new Tuple2>(checkpoint.checkpointId, ids)); + deque.addLast(new Tuple2>(checkpoint.checkpointId, ids)); } return deque; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java index ab21586e440ba..604755d456df8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MessageAcknowledgingSourceBase.java @@ -96,13 +96,13 @@ public abstract class MessageAcknowledgingSourceBase private final TypeSerializer idSerializer; /** The list gathering the IDs of messages emitted during the current checkpoint. */ - private transient List idsForCurrentCheckpoint; + private transient Set idsForCurrentCheckpoint; /** * The list with IDs from checkpoints that were triggered, but not yet completed or notified of * completion. */ - protected transient ArrayDeque>> pendingCheckpoints; + protected transient ArrayDeque>> pendingCheckpoints; /** * Set which contain all processed ids. Ids are acknowledged after checkpoints. When restoring @@ -142,7 +142,7 @@ public void initializeState(FunctionInitializationContext context) throws Except .getOperatorStateStore() .getSerializableListState("message-acknowledging-source-state"); - this.idsForCurrentCheckpoint = new ArrayList<>(64); + this.idsForCurrentCheckpoint = new HashSet<>(64); this.pendingCheckpoints = new ArrayDeque<>(); this.idsProcessedButNotAcknowledged = new HashSet<>(); @@ -161,7 +161,7 @@ public void initializeState(FunctionInitializationContext context) throws Except pendingCheckpoints = SerializedCheckpointData.toDeque(retrievedStates.get(0), idSerializer); // build a set which contains all processed ids. It may be used to check if we have // already processed an incoming message. - for (Tuple2> checkpoint : pendingCheckpoints) { + for (Tuple2> checkpoint : pendingCheckpoints) { idsProcessedButNotAcknowledged.addAll(checkpoint.f1); } } else { @@ -185,7 +185,7 @@ public void close() throws Exception { * * @param uIds The list od IDs to acknowledge. */ - protected abstract void acknowledgeIDs(long checkpointId, List uIds); + protected abstract void acknowledgeIDs(long checkpointId, Set uIds); /** * Adds an ID to be stored with the current checkpoint. @@ -213,7 +213,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } pendingCheckpoints.addLast(new Tuple2<>(context.getCheckpointId(), idsForCurrentCheckpoint)); - idsForCurrentCheckpoint = new ArrayList<>(64); + idsForCurrentCheckpoint = new HashSet<>(64); this.checkpointedState.clear(); this.checkpointedState.add(SerializedCheckpointData.fromDeque(pendingCheckpoints, idSerializer)); @@ -223,8 +223,8 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { public void notifyCheckpointComplete(long checkpointId) throws Exception { LOG.debug("Committing Messages externally for checkpoint {}", checkpointId); - for (Iterator>> iter = pendingCheckpoints.iterator(); iter.hasNext();) { - Tuple2> checkpoint = iter.next(); + for (Iterator>> iter = pendingCheckpoints.iterator(); iter.hasNext();) { + Tuple2> checkpoint = iter.next(); long id = checkpoint.f0; if (id <= checkpointId) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MultipleIdsMessageAcknowledgingSourceBase.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MultipleIdsMessageAcknowledgingSourceBase.java index e7cdb99c24c3c..d0c0741ec74f7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MultipleIdsMessageAcknowledgingSourceBase.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/MultipleIdsMessageAcknowledgingSourceBase.java @@ -32,6 +32,7 @@ import java.util.Deque; import java.util.Iterator; import java.util.List; +import java.util.Set; /** * Abstract base class for data sources that receive elements from a message queue and @@ -110,7 +111,8 @@ public void close() throws Exception { * means of de-duplicating messages when the acknowledgment after a checkpoint * fails. */ - protected final void acknowledgeIDs(long checkpointId, List uniqueIds) { + @Override + protected final void acknowledgeIDs(long checkpointId, Set uniqueIds) { LOG.debug("Acknowledging ids for checkpoint {}", checkpointId); Iterator>> iterator = sessionIdsPerSnapshot.iterator(); while (iterator.hasNext()) { From bbac4a6c922199db08a5244d0fa1262a5f16d479 Mon Sep 17 00:00:00 2001 From: "Wright, Eron" Date: Wed, 16 Aug 2017 14:30:24 -0700 Subject: [PATCH 029/129] [FLINK-6630] [FLINK-6631] Implement FLIP-6 Mesos cluster entrypoints + MesosTaskExecutorRunner - bin: new entrypoints scripts for flip-6 - ClusterEntrypoint: Refactor the shutdown method - ClusterEntrypoint: Install default FileSystem (for parity with legacy entrypoints) - ClusterEntrypoint: new MesosJobClusterEntrypoint, MesosSessionClusterEntrypoint, MesosEntrypointUtils, MesosTaskExecutorRunner - MesosServices: enhanced with artifactServer, localActorSystem - MesosResourceManager: Fallback to old TM params when UNKNOWN resource profile is provided - MesosResourceManager: config setting for taskmanager startup script (mesos.resourcemanager.tasks.taskmanager-cmd) - test: added a 'noop' job graph for testing purposes This closes #4555. --- .../mesos-bin/mesos-appmaster-flip6-job.sh | 47 ++++ .../mesos-appmaster-flip6-session.sh | 47 ++++ .../mesos-bin/mesos-taskmanager-flip6.sh | 45 ++++ .../entrypoint/MesosEntrypointUtils.java | 177 +++++++++++++++ .../entrypoint/MesosJobClusterEntrypoint.java | 204 ++++++++++++++++++ .../MesosSessionClusterEntrypoint.java | 178 +++++++++++++++ .../entrypoint/MesosTaskExecutorRunner.java | 132 ++++++++++++ .../LaunchableMesosWorker.java | 30 ++- .../MesosApplicationMasterRunner.java | 131 +---------- .../MesosResourceManager.java | 47 ++-- .../MesosTaskManagerParameters.java | 20 +- .../services/AbstractMesosServices.java | 73 +++++++ .../services/MesosServices.java | 17 ++ .../services/MesosServicesUtils.java | 28 ++- .../services/StandaloneMesosServices.java | 10 +- .../services/ZooKeeperMesosServices.java | 30 ++- .../MesosFlinkResourceManagerTest.java | 1 + .../MesosResourceManagerTest.java | 48 +++-- .../runtime/entrypoint/ClusterEntrypoint.java | 27 +++ .../entrypoint/JobClusterEntrypoint.java | 10 +- .../entrypoint/SessionClusterEntrypoint.java | 8 +- .../runtime/entrypoint/StreamingNoop.java | 60 ++++++ 22 files changed, 1186 insertions(+), 184 deletions(-) create mode 100755 flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster-flip6-job.sh create mode 100755 flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster-flip6-session.sh create mode 100755 flink-dist/src/main/flink-bin/mesos-bin/mesos-taskmanager-flip6.sh create mode 100755 flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java create mode 100755 flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java create mode 100755 flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java mode change 100644 => 100755 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java create mode 100644 flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/AbstractMesosServices.java mode change 100644 => 100755 flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/ClusterEntrypoint.java create mode 100644 flink-tests/src/test/java/org/apache/flink/test/runtime/entrypoint/StreamingNoop.java diff --git a/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster-flip6-job.sh b/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster-flip6-job.sh new file mode 100755 index 0000000000000..b21670a6452ea --- /dev/null +++ b/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster-flip6-job.sh @@ -0,0 +1,47 @@ +#!/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. +################################################################################ + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +# get Flink config +. "$bin"/config.sh + +if [ "$FLINK_IDENT_STRING" = "" ]; then + FLINK_IDENT_STRING="$USER" +fi + +CC_CLASSPATH=`manglePathList $(constructFlinkClassPath):$INTERNAL_HADOOP_CLASSPATHS` + +log="${FLINK_LOG_DIR}/flink-${FLINK_IDENT_STRING}-mesos-appmaster-${HOSTNAME}.log" +log_setting="-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4j.properties -Dlogback.configurationFile=file:"$FLINK_CONF_DIR"/logback.xml" + +export FLINK_CONF_DIR +export FLINK_BIN_DIR +export FLINK_LIB_DIR + +exec $JAVA_RUN $JVM_ARGS -classpath "$CC_CLASSPATH" $log_setting org.apache.flink.mesos.entrypoint.MesosJobClusterEntrypoint "$@" + +rc=$? + +if [[ $rc -ne 0 ]]; then + echo "Error while starting the mesos application master. Please check ${log} for more details." +fi + +exit $rc diff --git a/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster-flip6-session.sh b/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster-flip6-session.sh new file mode 100755 index 0000000000000..b9e0f5375c640 --- /dev/null +++ b/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster-flip6-session.sh @@ -0,0 +1,47 @@ +#!/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. +################################################################################ + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +# get Flink config +. "$bin"/config.sh + +if [ "$FLINK_IDENT_STRING" = "" ]; then + FLINK_IDENT_STRING="$USER" +fi + +CC_CLASSPATH=`manglePathList $(constructFlinkClassPath):$INTERNAL_HADOOP_CLASSPATHS` + +log="${FLINK_LOG_DIR}/flink-${FLINK_IDENT_STRING}-mesos-appmaster-${HOSTNAME}.log" +log_setting="-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4j.properties -Dlogback.configurationFile=file:"$FLINK_CONF_DIR"/logback.xml" + +export FLINK_CONF_DIR +export FLINK_BIN_DIR +export FLINK_LIB_DIR + +exec $JAVA_RUN $JVM_ARGS -classpath "$CC_CLASSPATH" $log_setting org.apache.flink.mesos.entrypoint.MesosSessionClusterEntrypoint "$@" + +rc=$? + +if [[ $rc -ne 0 ]]; then + echo "Error while starting the mesos application master. Please check ${log} for more details." +fi + +exit $rc diff --git a/flink-dist/src/main/flink-bin/mesos-bin/mesos-taskmanager-flip6.sh b/flink-dist/src/main/flink-bin/mesos-bin/mesos-taskmanager-flip6.sh new file mode 100755 index 0000000000000..f2514429fe7e8 --- /dev/null +++ b/flink-dist/src/main/flink-bin/mesos-bin/mesos-taskmanager-flip6.sh @@ -0,0 +1,45 @@ +#!/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. +################################################################################ + +bin=`dirname "$0"` +bin=`cd "$bin"; pwd` + +# get Flink config +. "$bin"/config.sh + +CC_CLASSPATH=`manglePathList $(constructFlinkClassPath):$INTERNAL_HADOOP_CLASSPATHS` + +log=flink-taskmanager.log +log_setting="-Dlog.file="$log" -Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4j.properties -Dlogback.configurationFile=file:"$FLINK_CONF_DIR"/logback.xml" + +# Add precomputed memory JVM options +if [ -z "${FLINK_ENV_JAVA_OPTS_MEM}" ]; then + FLINK_ENV_JAVA_OPTS_MEM="" +fi +export FLINK_ENV_JAVA_OPTS="${FLINK_ENV_JAVA_OPTS} ${FLINK_ENV_JAVA_OPTS_MEM}" + +# Add TaskManager-specific JVM options +export FLINK_ENV_JAVA_OPTS="${FLINK_ENV_JAVA_OPTS} ${FLINK_ENV_JAVA_OPTS_TM}" + +export FLINK_CONF_DIR +export FLINK_BIN_DIR +export FLINK_LIB_DIR + +exec $JAVA_RUN $JVM_ARGS ${FLINK_ENV_JAVA_OPTS} -classpath "$CC_CLASSPATH" $log_setting org.apache.flink.mesos.entrypoint.MesosTaskExecutorRunner "$@" + diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java new file mode 100755 index 0000000000000..0d81eadf71bac --- /dev/null +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java @@ -0,0 +1,177 @@ +/* + * 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.mesos.entrypoint; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.mesos.configuration.MesosOptions; +import org.apache.flink.mesos.runtime.clusterframework.MesosTaskManagerParameters; +import org.apache.flink.mesos.util.MesosConfiguration; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.ContainerSpecification; +import org.apache.flink.runtime.clusterframework.overlays.CompositeContainerOverlay; +import org.apache.flink.runtime.clusterframework.overlays.FlinkDistributionOverlay; +import org.apache.flink.runtime.clusterframework.overlays.HadoopConfOverlay; +import org.apache.flink.runtime.clusterframework.overlays.HadoopUserOverlay; +import org.apache.flink.runtime.clusterframework.overlays.KeytabOverlay; +import org.apache.flink.runtime.clusterframework.overlays.Krb5ConfOverlay; +import org.apache.flink.runtime.clusterframework.overlays.SSLStoreOverlay; + +import org.apache.commons.cli.CommandLine; +import org.apache.mesos.Protos; +import org.slf4j.Logger; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import scala.concurrent.duration.Duration; +import scala.concurrent.duration.FiniteDuration; + +/** + * Utils for Mesos entrpoints. + */ +public class MesosEntrypointUtils { + + /** + * Loads the global configuration and adds the dynamic properties parsed from + * the given command line. + * + * @param cmd command line to parse for dynamic properties + * @return Global configuration with dynamic properties set + * @deprecated replace once FLINK-7269 has been merged + */ + @Deprecated + public static Configuration loadConfiguration(CommandLine cmd) { + + // merge the dynamic properties from the command-line + Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); + GlobalConfiguration.setDynamicProperties(dynamicProperties); + Configuration config = GlobalConfiguration.loadConfiguration(); + + return config; + } + + /** + * Loads and validates the Mesos scheduler configuration. + * @param flinkConfig the global configuration. + * @param hostname the hostname to advertise to the Mesos master. + */ + public static MesosConfiguration createMesosSchedulerConfiguration(Configuration flinkConfig, String hostname) { + + Protos.FrameworkInfo.Builder frameworkInfo = Protos.FrameworkInfo.newBuilder() + .setHostname(hostname); + Protos.Credential.Builder credential = null; + + if (!flinkConfig.contains(MesosOptions.MASTER_URL)) { + throw new IllegalConfigurationException(MesosOptions.MASTER_URL.key() + " must be configured."); + } + String masterUrl = flinkConfig.getString(MesosOptions.MASTER_URL); + + Duration failoverTimeout = FiniteDuration.apply( + flinkConfig.getInteger( + MesosOptions.FAILOVER_TIMEOUT_SECONDS), + TimeUnit.SECONDS); + frameworkInfo.setFailoverTimeout(failoverTimeout.toSeconds()); + + frameworkInfo.setName(flinkConfig.getString( + MesosOptions.RESOURCEMANAGER_FRAMEWORK_NAME)); + + frameworkInfo.setRole(flinkConfig.getString( + MesosOptions.RESOURCEMANAGER_FRAMEWORK_ROLE)); + + frameworkInfo.setUser(flinkConfig.getString( + MesosOptions.RESOURCEMANAGER_FRAMEWORK_USER)); + + if (flinkConfig.contains(MesosOptions.RESOURCEMANAGER_FRAMEWORK_PRINCIPAL)) { + frameworkInfo.setPrincipal(flinkConfig.getString( + MesosOptions.RESOURCEMANAGER_FRAMEWORK_PRINCIPAL)); + + credential = Protos.Credential.newBuilder(); + credential.setPrincipal(frameworkInfo.getPrincipal()); + + // some environments use a side-channel to communicate the secret to Mesos, + // and thus don't set the 'secret' configuration setting + if (flinkConfig.contains(MesosOptions.RESOURCEMANAGER_FRAMEWORK_SECRET)) { + credential.setSecret(flinkConfig.getString( + MesosOptions.RESOURCEMANAGER_FRAMEWORK_SECRET)); + } + } + + MesosConfiguration mesos = + new MesosConfiguration(masterUrl, frameworkInfo, scala.Option.apply(credential)); + + return mesos; + } + + public static MesosTaskManagerParameters createTmParameters(Configuration configuration, Logger log) { + // TM configuration + final MesosTaskManagerParameters taskManagerParameters = MesosTaskManagerParameters.create(configuration); + + log.info("TaskManagers will be created with {} task slots", + taskManagerParameters.containeredParameters().numSlots()); + log.info("TaskManagers will be started with container size {} MB, JVM heap size {} MB, " + + "JVM direct memory limit {} MB, {} cpus", + taskManagerParameters.containeredParameters().taskManagerTotalMemoryMB(), + taskManagerParameters.containeredParameters().taskManagerHeapSizeMB(), + taskManagerParameters.containeredParameters().taskManagerDirectMemoryLimitMB(), + taskManagerParameters.cpus()); + + return taskManagerParameters; + } + + public static ContainerSpecification createContainerSpec(Configuration configuration, Configuration dynamicProperties) + throws Exception { + // generate a container spec which conveys the artifacts/vars needed to launch a TM + ContainerSpecification spec = new ContainerSpecification(); + + // propagate the AM dynamic configuration to the TM + spec.getDynamicConfiguration().addAll(dynamicProperties); + + applyOverlays(configuration, spec); + + return spec; + } + + /** + * Generate a container specification as a TaskManager template. + * + *

This code is extremely Mesos-specific and registers all the artifacts that the TaskManager + * needs (such as JAR file, config file, ...) and all environment variables into a container specification. + * The Mesos fetcher then ensures that those artifacts will be copied into the task's sandbox directory. + * A lightweight HTTP server serves the artifacts to the fetcher. + */ + public static void applyOverlays( + Configuration configuration, ContainerSpecification containerSpec) throws IOException { + + // create the overlays that will produce the specification + CompositeContainerOverlay overlay = new CompositeContainerOverlay( + FlinkDistributionOverlay.newBuilder().fromEnvironment(configuration).build(), + HadoopConfOverlay.newBuilder().fromEnvironment(configuration).build(), + HadoopUserOverlay.newBuilder().fromEnvironment(configuration).build(), + KeytabOverlay.newBuilder().fromEnvironment(configuration).build(), + Krb5ConfOverlay.newBuilder().fromEnvironment(configuration).build(), + SSLStoreOverlay.newBuilder().fromEnvironment(configuration).build() + ); + + // apply the overlays + overlay.configure(containerSpec); + } + +} diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java new file mode 100755 index 0000000000000..890c4a790c072 --- /dev/null +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -0,0 +1,204 @@ +/* + * 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.mesos.entrypoint; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.mesos.runtime.clusterframework.MesosResourceManager; +import org.apache.flink.mesos.runtime.clusterframework.MesosTaskManagerParameters; +import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; +import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; +import org.apache.flink.mesos.util.MesosConfiguration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.ContainerSpecification; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.JobClusterEntrypoint; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.PosixParser; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.ObjectInputStream; + +/** + * Entry point for Mesos per-job clusters. + */ +public class MesosJobClusterEntrypoint extends JobClusterEntrypoint { + + public static final String JOB_GRAPH_FILE_PATH = "flink.jobgraph.path"; + + // ------------------------------------------------------------------------ + // Command-line options + // ------------------------------------------------------------------------ + + private static final Options ALL_OPTIONS; + + static { + ALL_OPTIONS = + new Options() + .addOption(BootstrapTools.newDynamicPropertiesOption()); + } + + private MesosConfiguration schedulerConfiguration; + + private MesosServices mesosServices; + + private MesosTaskManagerParameters taskManagerParameters; + + private ContainerSpecification taskManagerContainerSpec; + + public MesosJobClusterEntrypoint(Configuration config) { + super(config); + } + + @Override + protected void initializeServices(Configuration config) throws Exception { + super.initializeServices(config); + + final String hostname = config.getString(JobManagerOptions.ADDRESS); + + // Mesos configuration + schedulerConfiguration = MesosEntrypointUtils.createMesosSchedulerConfiguration(config, hostname); + + // services + mesosServices = MesosServicesUtils.createMesosServices(config, hostname); + + // TM configuration + taskManagerParameters = MesosEntrypointUtils.createTmParameters(config, LOG); + taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, GlobalConfiguration.getDynamicProperties()); + } + + @Override + protected void startClusterComponents(Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry) throws Exception { + super.startClusterComponents(configuration, rpcService, highAvailabilityServices, blobServer, heartbeatServices, metricRegistry); + } + + @Override + protected ResourceManager createResourceManager( + Configuration configuration, + ResourceID resourceId, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + MetricRegistry metricRegistry, + FatalErrorHandler fatalErrorHandler) throws Exception { + final ResourceManagerConfiguration rmConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration); + final ResourceManagerRuntimeServicesConfiguration rmServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration); + final ResourceManagerRuntimeServices rmRuntimeServices = ResourceManagerRuntimeServices.fromConfiguration( + rmServicesConfiguration, + highAvailabilityServices, + rpcService.getScheduledExecutor()); + + return new MesosResourceManager( + rpcService, + ResourceManager.RESOURCE_MANAGER_NAME, + resourceId, + rmConfiguration, + highAvailabilityServices, + heartbeatServices, + rmRuntimeServices.getSlotManager(), + metricRegistry, + rmRuntimeServices.getJobLeaderIdService(), + fatalErrorHandler, + configuration, + mesosServices, + schedulerConfiguration, + taskManagerParameters, + taskManagerContainerSpec + ); + } + + @Override + protected JobGraph retrieveJobGraph(Configuration configuration) throws FlinkException { + String jobGraphFile = configuration.getString(JOB_GRAPH_FILE_PATH, "job.graph"); + File fp = new File(jobGraphFile); + + try (FileInputStream input = new FileInputStream(fp); + ObjectInputStream obInput = new ObjectInputStream(input)) { + + return (JobGraph) obInput.readObject(); + } catch (FileNotFoundException e) { + throw new FlinkException("Could not find the JobGraph file.", e); + } catch (ClassNotFoundException | IOException e) { + throw new FlinkException("Could not load the JobGraph from file.", e); + } + } + + @Override + protected void stopClusterComponents(boolean cleanupHaData) throws Exception { + Throwable exception = null; + + try { + super.stopClusterComponents(cleanupHaData); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + + if (mesosServices != null) { + try { + mesosServices.close(cleanupHaData); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + + if (exception != null) { + throw new FlinkException("Could not properly shut down the Mesos job cluster entry point.", exception); + } + } + + public static void main(String[] args) { + // load configuration incl. dynamic properties + CommandLineParser parser = new PosixParser(); + CommandLine cmd; + try { + cmd = parser.parse(ALL_OPTIONS, args); + } + catch (Exception e){ + LOG.error("Could not parse the command-line options.", e); + System.exit(STARTUP_FAILURE_RETURN_CODE); + return; + } + + Configuration configuration = MesosEntrypointUtils.loadConfiguration(cmd); + + MesosJobClusterEntrypoint clusterEntrypoint = new MesosJobClusterEntrypoint(configuration); + + clusterEntrypoint.startCluster(); + } +} diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java new file mode 100755 index 0000000000000..67f5899c392bf --- /dev/null +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.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.mesos.entrypoint; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.mesos.runtime.clusterframework.MesosResourceManager; +import org.apache.flink.mesos.runtime.clusterframework.MesosTaskManagerParameters; +import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; +import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; +import org.apache.flink.mesos.util.MesosConfiguration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.ContainerSpecification; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.entrypoint.SessionClusterEntrypoint; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.resourcemanager.ResourceManager; +import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServices; +import org.apache.flink.runtime.resourcemanager.ResourceManagerRuntimeServicesConfiguration; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.FlinkException; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.PosixParser; + +/** + * Entry point for Mesos session clusters. + */ +public class MesosSessionClusterEntrypoint extends SessionClusterEntrypoint { + + // ------------------------------------------------------------------------ + // Command-line options + // ------------------------------------------------------------------------ + + private static final Options ALL_OPTIONS; + + static { + ALL_OPTIONS = + new Options() + .addOption(BootstrapTools.newDynamicPropertiesOption()); + } + + private MesosConfiguration mesosConfig; + + private MesosServices mesosServices; + + private MesosTaskManagerParameters taskManagerParameters; + + private ContainerSpecification taskManagerContainerSpec; + + public MesosSessionClusterEntrypoint(Configuration config) { + super(config); + } + + @Override + protected void initializeServices(Configuration config) throws Exception { + super.initializeServices(config); + + final String hostname = config.getString(JobManagerOptions.ADDRESS); + + // Mesos configuration + mesosConfig = MesosEntrypointUtils.createMesosSchedulerConfiguration(config, hostname); + + // services + mesosServices = MesosServicesUtils.createMesosServices(config, hostname); + + // TM configuration + taskManagerParameters = MesosEntrypointUtils.createTmParameters(config, LOG); + taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, GlobalConfiguration.getDynamicProperties()); + } + + @Override + protected void startClusterComponents(Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, BlobServer blobServer, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry) throws Exception { + super.startClusterComponents(configuration, rpcService, highAvailabilityServices, blobServer, heartbeatServices, metricRegistry); + } + + @Override + protected ResourceManager createResourceManager( + Configuration configuration, + ResourceID resourceId, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + MetricRegistry metricRegistry, + FatalErrorHandler fatalErrorHandler) throws Exception { + final ResourceManagerConfiguration rmConfiguration = ResourceManagerConfiguration.fromConfiguration(configuration); + final ResourceManagerRuntimeServicesConfiguration rmServicesConfiguration = ResourceManagerRuntimeServicesConfiguration.fromConfiguration(configuration); + final ResourceManagerRuntimeServices rmRuntimeServices = ResourceManagerRuntimeServices.fromConfiguration( + rmServicesConfiguration, + highAvailabilityServices, + rpcService.getScheduledExecutor()); + + return new MesosResourceManager( + rpcService, + ResourceManager.RESOURCE_MANAGER_NAME, + resourceId, + rmConfiguration, + highAvailabilityServices, + heartbeatServices, + rmRuntimeServices.getSlotManager(), + metricRegistry, + rmRuntimeServices.getJobLeaderIdService(), + fatalErrorHandler, + configuration, + mesosServices, + mesosConfig, + taskManagerParameters, + taskManagerContainerSpec + ); + } + + @Override + protected void stopClusterComponents(boolean cleanupHaData) throws Exception { + Throwable exception = null; + + try { + super.stopClusterComponents(cleanupHaData); + } catch (Throwable t) { + exception = t; + } + + if (mesosServices != null) { + try { + mesosServices.close(cleanupHaData); + } catch (Throwable t) { + exception = t; + } + } + + if (exception != null) { + throw new FlinkException("Could not properly shut down the Mesos session cluster entry point.", exception); + } + } + + public static void main(String[] args) { + // load configuration incl. dynamic properties + CommandLineParser parser = new PosixParser(); + CommandLine cmd; + try { + cmd = parser.parse(ALL_OPTIONS, args); + } + catch (Exception e){ + LOG.error("Could not parse the command-line options.", e); + System.exit(STARTUP_FAILURE_RETURN_CODE); + return; + } + + Configuration configuration = MesosEntrypointUtils.loadConfiguration(cmd); + + MesosSessionClusterEntrypoint clusterEntrypoint = new MesosSessionClusterEntrypoint(configuration); + + clusterEntrypoint.startCluster(); + } +} diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java new file mode 100644 index 0000000000000..c4343d27886d3 --- /dev/null +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java @@ -0,0 +1,132 @@ +/* + * 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.mesos.entrypoint; + +import org.apache.flink.configuration.AkkaOptions; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.mesos.runtime.clusterframework.MesosConfigKeys; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.security.SecurityUtils; +import org.apache.flink.runtime.taskexecutor.TaskManagerRunner; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.runtime.util.JvmShutdownSafeguard; +import org.apache.flink.runtime.util.SignalHandler; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.PosixParser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Callable; + +/** + * The entry point for running a TaskManager in a Mesos container. + */ +public class MesosTaskExecutorRunner { + + private static final Logger LOG = LoggerFactory.getLogger(MesosTaskExecutorRunner.class); + + private static final int INIT_ERROR_EXIT_CODE = 31; + + private static final Options ALL_OPTIONS; + + static { + ALL_OPTIONS = + new Options() + .addOption(BootstrapTools.newDynamicPropertiesOption()); + } + + public static void main(String[] args) throws Exception { + EnvironmentInformation.logEnvironmentInfo(LOG, MesosTaskExecutorRunner.class.getSimpleName(), args); + SignalHandler.register(LOG); + JvmShutdownSafeguard.installAsShutdownHook(LOG); + + // try to parse the command line arguments + CommandLineParser parser = new PosixParser(); + CommandLine cmd = parser.parse(ALL_OPTIONS, args); + + final Configuration configuration; + try { + configuration = MesosEntrypointUtils.loadConfiguration(cmd); + } + catch (Throwable t) { + LOG.error("Failed to load the TaskManager configuration and dynamic properties.", t); + System.exit(INIT_ERROR_EXIT_CODE); + return; + } + + // read the environment variables + final Map envs = System.getenv(); + final String tmpDirs = envs.get(MesosConfigKeys.ENV_FLINK_TMP_DIR); + + // configure local directory + String flinkTempDirs = configuration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, null); + if (flinkTempDirs != null) { + LOG.info("Overriding Mesos temporary file directories with those " + + "specified in the Flink config: {}", flinkTempDirs); + } + else if (tmpDirs != null) { + LOG.info("Setting directories for temporary files to: {}", tmpDirs); + configuration.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, tmpDirs); + } + + // configure the default filesystem + try { + FileSystem.setDefaultScheme(configuration); + } catch (IOException e) { + throw new IOException("Error while setting the default " + + "filesystem scheme from configuration.", e); + } + + // tell akka to die in case of an error + configuration.setBoolean(AkkaOptions.JVM_EXIT_ON_FATAL_ERROR, true); + + // Infer the resource identifier from the environment variable + String containerID = Preconditions.checkNotNull(envs.get(MesosConfigKeys.ENV_FLINK_CONTAINER_ID)); + final ResourceID resourceId = new ResourceID(containerID); + LOG.info("ResourceID assigned for this container: {}", resourceId); + + // Run the TM in the security context + SecurityUtils.SecurityConfiguration sc = new SecurityUtils.SecurityConfiguration(configuration); + SecurityUtils.install(sc); + + try { + SecurityUtils.getInstalledContext().runSecured(new Callable() { + @Override + public Integer call() throws Exception { + TaskManagerRunner.runTaskManager(configuration, resourceId); + + return 0; + } + }); + } + catch (Throwable t) { + LOG.error("Error while starting the TaskManager", t); + System.exit(INIT_ERROR_EXIT_CODE); + } + } +} diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java index ce7bb9d6271fc..2c3250738027c 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java @@ -23,6 +23,7 @@ import org.apache.flink.mesos.Utils; import org.apache.flink.mesos.scheduler.LaunchableTask; import org.apache.flink.mesos.util.MesosArtifactResolver; +import org.apache.flink.mesos.util.MesosArtifactServer; import org.apache.flink.mesos.util.MesosConfiguration; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; @@ -36,6 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Map; @@ -261,12 +263,15 @@ public Protos.TaskInfo launch(Protos.SlaveID slaveId, TaskAssignmentResult assig env.addVariables(variable(MesosConfigKeys.ENV_FRAMEWORK_NAME, mesosConfiguration.frameworkInfo().getName())); // build the launch command w/ dynamic application properties - Option bootstrapCmdOption = params.bootstrapCommand(); - - final String bootstrapCommand = bootstrapCmdOption.isDefined() ? bootstrapCmdOption.get() + " && " : ""; - final String launchCommand = bootstrapCommand + "$FLINK_HOME/bin/mesos-taskmanager.sh " + ContainerSpecification.formatSystemProperties(dynamicProperties); - - cmd.setValue(launchCommand); + StringBuilder launchCommand = new StringBuilder(); + if (params.bootstrapCommand().isDefined()) { + launchCommand.append(params.bootstrapCommand().get()).append(" && "); + } + launchCommand + .append(params.command()) + .append(" ") + .append(ContainerSpecification.formatSystemProperties(dynamicProperties)); + cmd.setValue(launchCommand.toString()); // build the container info Protos.ContainerInfo.Builder containerInfo = Protos.ContainerInfo.newBuilder(); @@ -312,4 +317,17 @@ public String toString() { "taskRequest=" + taskRequest + '}'; } + + /** + * Configures an artifact server to serve the artifacts associated with a container specification. + * @param server the server to configure. + * @param container the container with artifacts to serve. + * @throws IOException if the artifacts cannot be accessed. + */ + static void configureArtifactServer(MesosArtifactServer server, ContainerSpecification container) throws IOException { + // serve the artifacts associated with the container environment + for (ContainerSpecification.Artifact artifact : container.getArtifacts()) { + server.addPath(artifact.source, artifact.dest); + } + } } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java old mode 100644 new mode 100755 index 7891386675e56..3d16a664a2268 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java @@ -21,11 +21,10 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.WebOptions; import org.apache.flink.core.fs.FileSystem; -import org.apache.flink.mesos.configuration.MesosOptions; +import org.apache.flink.mesos.entrypoint.MesosEntrypointUtils; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.services.MesosServicesUtils; import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore; @@ -34,13 +33,6 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; -import org.apache.flink.runtime.clusterframework.overlays.CompositeContainerOverlay; -import org.apache.flink.runtime.clusterframework.overlays.FlinkDistributionOverlay; -import org.apache.flink.runtime.clusterframework.overlays.HadoopConfOverlay; -import org.apache.flink.runtime.clusterframework.overlays.HadoopUserOverlay; -import org.apache.flink.runtime.clusterframework.overlays.KeytabOverlay; -import org.apache.flink.runtime.clusterframework.overlays.Krb5ConfOverlay; -import org.apache.flink.runtime.clusterframework.overlays.SSLStoreOverlay; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.jobmanager.JobManager; @@ -65,7 +57,6 @@ import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.Options; import org.apache.commons.cli.PosixParser; -import org.apache.mesos.Protos; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,7 +64,6 @@ import java.net.InetAddress; import java.net.URL; import java.util.Map; -import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -81,7 +71,6 @@ import java.util.concurrent.TimeUnit; import scala.Option; -import scala.concurrent.duration.Duration; import scala.concurrent.duration.FiniteDuration; import static org.apache.flink.util.Preconditions.checkState; @@ -222,7 +211,7 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie LOG.info("App Master Hostname to use: {}", appMasterHostname); // Mesos configuration - final MesosConfiguration mesosConfig = createMesosConfig(config, appMasterHostname); + final MesosConfiguration mesosConfig = MesosEntrypointUtils.createMesosSchedulerConfiguration(config, appMasterHostname); // JM configuration int numberProcessors = Hardware.getNumberCPUCores(); @@ -235,19 +224,10 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie numberProcessors, new ExecutorThreadFactory("mesos-jobmanager-io")); - mesosServices = MesosServicesUtils.createMesosServices(config); + mesosServices = MesosServicesUtils.createMesosServices(config, appMasterHostname); // TM configuration - final MesosTaskManagerParameters taskManagerParameters = MesosTaskManagerParameters.create(config); - - LOG.info("TaskManagers will be created with {} task slots", - taskManagerParameters.containeredParameters().numSlots()); - LOG.info("TaskManagers will be started with container size {} MB, JVM heap size {} MB, " + - "JVM direct memory limit {} MB, {} cpus", - taskManagerParameters.containeredParameters().taskManagerTotalMemoryMB(), - taskManagerParameters.containeredParameters().taskManagerHeapSizeMB(), - taskManagerParameters.containeredParameters().taskManagerDirectMemoryLimitMB(), - taskManagerParameters.cpus()); + final MesosTaskManagerParameters taskManagerParameters = MesosEntrypointUtils.createTmParameters(config, LOG); // JM endpoint, which should be explicitly configured based on acquired net resources final int listeningPort = config.getInteger(JobManagerOptions.PORT); @@ -268,9 +248,7 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie // try to start the artifact server LOG.debug("Starting Artifact Server"); - final int artifactServerPort = config.getInteger(MesosOptions.ARTIFACT_SERVER_PORT); - final String artifactServerPrefix = UUID.randomUUID().toString(); - artifactServer = new MesosArtifactServer(artifactServerPrefix, akkaHostname, artifactServerPort, config); + artifactServer = mesosServices.getArtifactServer(); // ----------------- (3) Generate the configuration for the TaskManagers ------------------- @@ -287,10 +265,10 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie taskManagerContainerSpec.getDynamicConfiguration().addAll(taskManagerConfig); // apply the overlays - applyOverlays(config, taskManagerContainerSpec); + MesosEntrypointUtils.applyOverlays(config, taskManagerContainerSpec); // configure the artifact server to serve the specified artifacts - configureArtifactServer(artifactServer, taskManagerContainerSpec); + LaunchableMesosWorker.configureArtifactServer(artifactServer, taskManagerContainerSpec); // ----------------- (4) start the actors ------------------- @@ -386,14 +364,6 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie } } - if (artifactServer != null) { - try { - artifactServer.stop(); - } catch (Throwable ignored) { - LOG.error("Failed to stop the artifact server", ignored); - } - } - if (actorSystem != null) { try { actorSystem.shutdown(); @@ -444,12 +414,6 @@ protected int runPrivileged(Configuration config, Configuration dynamicPropertie } } - try { - artifactServer.stop(); - } catch (Throwable t) { - LOG.error("Failed to stop the artifact server", t); - } - if (highAvailabilityServices != null) { try { highAvailabilityServices.close(); @@ -490,85 +454,4 @@ protected Class getArchivistClass() { return MemoryArchivist.class; } - /** - * Loads and validates the ResourceManager Mesos configuration from the given Flink configuration. - */ - public static MesosConfiguration createMesosConfig(Configuration flinkConfig, String hostname) { - - Protos.FrameworkInfo.Builder frameworkInfo = Protos.FrameworkInfo.newBuilder() - .setHostname(hostname); - Protos.Credential.Builder credential = null; - - if (!flinkConfig.contains(MesosOptions.MASTER_URL)) { - throw new IllegalConfigurationException(MesosOptions.MASTER_URL.key() + " must be configured."); - } - String masterUrl = flinkConfig.getString(MesosOptions.MASTER_URL); - - Duration failoverTimeout = FiniteDuration.apply( - flinkConfig.getInteger( - MesosOptions.FAILOVER_TIMEOUT_SECONDS), - TimeUnit.SECONDS); - frameworkInfo.setFailoverTimeout(failoverTimeout.toSeconds()); - - frameworkInfo.setName(flinkConfig.getString( - MesosOptions.RESOURCEMANAGER_FRAMEWORK_NAME)); - - frameworkInfo.setRole(flinkConfig.getString( - MesosOptions.RESOURCEMANAGER_FRAMEWORK_ROLE)); - - frameworkInfo.setUser(flinkConfig.getString( - MesosOptions.RESOURCEMANAGER_FRAMEWORK_USER)); - - if (flinkConfig.contains(MesosOptions.RESOURCEMANAGER_FRAMEWORK_PRINCIPAL)) { - frameworkInfo.setPrincipal(flinkConfig.getString( - MesosOptions.RESOURCEMANAGER_FRAMEWORK_PRINCIPAL)); - - credential = Protos.Credential.newBuilder(); - credential.setPrincipal(frameworkInfo.getPrincipal()); - - // some environments use a side-channel to communicate the secret to Mesos, - // and thus don't set the 'secret' configuration setting - if (flinkConfig.contains(MesosOptions.RESOURCEMANAGER_FRAMEWORK_SECRET)) { - credential.setSecret(flinkConfig.getString( - MesosOptions.RESOURCEMANAGER_FRAMEWORK_SECRET)); - } - } - - MesosConfiguration mesos = - new MesosConfiguration(masterUrl, frameworkInfo, scala.Option.apply(credential)); - - return mesos; - } - - /** - * Generate a container specification as a TaskManager template. - * - *

This code is extremely Mesos-specific and registers all the artifacts that the TaskManager - * needs (such as JAR file, config file, ...) and all environment variables into a container specification. - * The Mesos fetcher then ensures that those artifacts will be copied into the task's sandbox directory. - * A lightweight HTTP server serves the artifacts to the fetcher. - */ - private static void applyOverlays( - Configuration globalConfiguration, ContainerSpecification containerSpec) throws IOException { - - // create the overlays that will produce the specification - CompositeContainerOverlay overlay = new CompositeContainerOverlay( - FlinkDistributionOverlay.newBuilder().fromEnvironment(globalConfiguration).build(), - HadoopConfOverlay.newBuilder().fromEnvironment(globalConfiguration).build(), - HadoopUserOverlay.newBuilder().fromEnvironment(globalConfiguration).build(), - KeytabOverlay.newBuilder().fromEnvironment(globalConfiguration).build(), - Krb5ConfOverlay.newBuilder().fromEnvironment(globalConfiguration).build(), - SSLStoreOverlay.newBuilder().fromEnvironment(globalConfiguration).build() - ); - - // apply the overlays - overlay.configure(containerSpec); - } - - private static void configureArtifactServer(MesosArtifactServer server, ContainerSpecification container) throws IOException { - // serve the artifacts associated with the container environment - for (ContainerSpecification.Artifact artifact : container.getArtifacts()) { - server.addPath(artifact.source, artifact.dest); - } - } } 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 736af59514d2e..445010bc71a3d 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 @@ -20,6 +20,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore; import org.apache.flink.mesos.scheduler.ConnectionMonitor; import org.apache.flink.mesos.scheduler.LaunchCoordinator; @@ -38,7 +39,7 @@ import org.apache.flink.mesos.scheduler.messages.ResourceOffers; import org.apache.flink.mesos.scheduler.messages.SlaveLost; import org.apache.flink.mesos.scheduler.messages.StatusUpdate; -import org.apache.flink.mesos.util.MesosArtifactResolver; +import org.apache.flink.mesos.util.MesosArtifactServer; import org.apache.flink.mesos.util.MesosConfiguration; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.ContainerSpecification; @@ -75,6 +76,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -98,17 +100,20 @@ public class MesosResourceManager extends ResourceManager(taskManagerParameters.containeredParameters().taskManagerEnv())), taskManagerParameters.containerVolumes(), taskManagerParameters.constraints(), + taskManagerParameters.command(), taskManagerParameters.bootstrapCommand(), taskManagerParameters.getTaskManagerHostname() ); + LOG.debug("LaunchableMesosWorker parameters: {}", params); + LaunchableMesosWorker launchable = new LaunchableMesosWorker( - artifactResolver, + artifactServer, params, taskManagerContainerSpec, taskID, diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java index f5a415e2ab243..3859913ecda3d 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java @@ -74,6 +74,10 @@ public class MesosTaskManagerParameters { key("mesos.resourcemanager.tasks.hostname") .noDefaultValue(); + public static final ConfigOption MESOS_TM_CMD = + key("mesos.resourcemanager.tasks.taskmanager-cmd") + .defaultValue("$FLINK_HOME/bin/mesos-taskmanager.sh"); // internal + public static final ConfigOption MESOS_TM_BOOTSTRAP_CMD = key("mesos.resourcemanager.tasks.bootstrap-cmd") .noDefaultValue(); @@ -107,6 +111,8 @@ public class MesosTaskManagerParameters { private final List constraints; + private final String command; + private final Option bootstrapCommand; private final Option taskManagerHostname; @@ -118,6 +124,7 @@ public MesosTaskManagerParameters( ContaineredTaskManagerParameters containeredParameters, List containerVolumes, List constraints, + String command, Option bootstrapCommand, Option taskManagerHostname) { @@ -127,6 +134,7 @@ public MesosTaskManagerParameters( this.containeredParameters = Preconditions.checkNotNull(containeredParameters); this.containerVolumes = Preconditions.checkNotNull(containerVolumes); this.constraints = Preconditions.checkNotNull(constraints); + this.command = Preconditions.checkNotNull(command); this.bootstrapCommand = Preconditions.checkNotNull(bootstrapCommand); this.taskManagerHostname = Preconditions.checkNotNull(taskManagerHostname); } @@ -182,6 +190,13 @@ public Option getTaskManagerHostname() { return taskManagerHostname; } + /** + * Get the command. + */ + public String command() { + return command; + } + /** * Get the bootstrap command. */ @@ -199,6 +214,7 @@ public String toString() { ", containerVolumes=" + containerVolumes + ", constraints=" + constraints + ", taskManagerHostName=" + taskManagerHostname + + ", command=" + command + ", bootstrapCommand=" + bootstrapCommand + '}'; } @@ -249,7 +265,8 @@ public static MesosTaskManagerParameters create(Configuration flinkConfig) { //obtain Task Manager Host Name from the configuration Option taskManagerHostname = Option.apply(flinkConfig.getString(MESOS_TM_HOSTNAME)); - //obtain bootstrap command from the configuration + //obtain command-line from the configuration + String tmCommand = flinkConfig.getString(MESOS_TM_CMD); Option tmBootstrapCommand = Option.apply(flinkConfig.getString(MESOS_TM_BOOTSTRAP_CMD)); return new MesosTaskManagerParameters( @@ -259,6 +276,7 @@ public static MesosTaskManagerParameters create(Configuration flinkConfig) { containeredParameters, containerVolumes, constraints, + tmCommand, tmBootstrapCommand, taskManagerHostname); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/AbstractMesosServices.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/AbstractMesosServices.java new file mode 100644 index 0000000000000..e4f4cf7ae4153 --- /dev/null +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/AbstractMesosServices.java @@ -0,0 +1,73 @@ +/* + * 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.mesos.runtime.clusterframework.services; + +import org.apache.flink.mesos.util.MesosArtifactServer; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; + +import akka.actor.ActorSystem; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * An abrstact implementation of {@link MesosServices}. + */ +public abstract class AbstractMesosServices implements MesosServices { + + private final ActorSystem actorSystem; + + private final MesosArtifactServer artifactServer; + + protected AbstractMesosServices(ActorSystem actorSystem, MesosArtifactServer artifactServer) { + this.actorSystem = checkNotNull(actorSystem); + this.artifactServer = checkNotNull(artifactServer); + } + + @Override + public ActorSystem getLocalActorSystem() { + return actorSystem; + } + + @Override + public MesosArtifactServer getArtifactServer() { + return artifactServer; + } + + @Override + public void close(boolean cleanup) throws Exception { + Throwable exception = null; + + try { + actorSystem.shutdown(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + + try { + artifactServer.stop(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + + if (exception != null) { + throw new FlinkException("Could not properly shut down the Mesos services.", exception); + } + } +} diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/MesosServices.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/MesosServices.java index 5655bfc9277f8..6a64f4f570ee4 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/MesosServices.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/MesosServices.java @@ -20,6 +20,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore; +import org.apache.flink.mesos.util.MesosArtifactServer; + +import akka.actor.ActorSystem; import java.util.concurrent.Executor; @@ -41,6 +44,20 @@ MesosWorkerStore createMesosWorkerStore( Configuration configuration, Executor executor) throws Exception; + /** + * Gets a local {@link ActorSystem} which is used for child actors within + * {@link org.apache.flink.mesos.runtime.clusterframework.MesosResourceManager}. + * + * @return a reference to an actor system. + */ + ActorSystem getLocalActorSystem(); + + /** + * Gets the artifact server with which to serve essential resources to task managers. + * @return a reference to an artifact server. + */ + MesosArtifactServer getArtifactServer(); + /** * Closes all state maintained by the mesos services implementation. * diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/MesosServicesUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/MesosServicesUtils.java index 370a760465a6e..c5a8516b00933 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/MesosServicesUtils.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/MesosServicesUtils.java @@ -20,9 +20,16 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.mesos.configuration.MesosOptions; +import org.apache.flink.mesos.util.MesosArtifactServer; +import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.zookeeper.ZooKeeperUtilityFactory; +import akka.actor.ActorSystem; + +import java.util.UUID; + /** * Utilities for the {@link MesosServices}. */ @@ -32,15 +39,21 @@ public class MesosServicesUtils { * Creates a {@link MesosServices} instance depending on the high availability settings. * * @param configuration containing the high availability settings + * @param hostname the hostname to advertise to remote clients * @return a mesos services instance * @throws Exception if the mesos services instance could not be created */ - public static MesosServices createMesosServices(Configuration configuration) throws Exception { + public static MesosServices createMesosServices(Configuration configuration, String hostname) throws Exception { + + ActorSystem localActorSystem = AkkaUtils.createLocalActorSystem(configuration); + + MesosArtifactServer artifactServer = createArtifactServer(configuration, hostname); + HighAvailabilityMode highAvailabilityMode = HighAvailabilityMode.fromConfig(configuration); switch (highAvailabilityMode) { case NONE: - return new StandaloneMesosServices(); + return new StandaloneMesosServices(localActorSystem, artifactServer); case ZOOKEEPER: final String zkMesosRootPath = configuration.getString( @@ -50,10 +63,19 @@ public static MesosServices createMesosServices(Configuration configuration) thr configuration, zkMesosRootPath); - return new ZooKeeperMesosServices(zooKeeperUtilityFactory); + return new ZooKeeperMesosServices(localActorSystem, artifactServer, zooKeeperUtilityFactory); default: throw new Exception("High availability mode " + highAvailabilityMode + " is not supported."); } } + + private static MesosArtifactServer createArtifactServer(Configuration configuration, String hostname) throws Exception { + final int artifactServerPort = configuration.getInteger(MesosOptions.ARTIFACT_SERVER_PORT, 0); + + // a random prefix is affixed to artifact URLs to ensure uniqueness in the Mesos fetcher cache + final String artifactServerPrefix = UUID.randomUUID().toString(); + + return new MesosArtifactServer(artifactServerPrefix, hostname, artifactServerPort, configuration); + } } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/StandaloneMesosServices.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/StandaloneMesosServices.java index aa3157fd94ec1..b93fd298686b4 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/StandaloneMesosServices.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/StandaloneMesosServices.java @@ -21,13 +21,20 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore; import org.apache.flink.mesos.runtime.clusterframework.store.StandaloneMesosWorkerStore; +import org.apache.flink.mesos.util.MesosArtifactServer; + +import akka.actor.ActorSystem; import java.util.concurrent.Executor; /** * {@link MesosServices} implementation for the standalone mode. */ -public class StandaloneMesosServices implements MesosServices { +public class StandaloneMesosServices extends AbstractMesosServices { + + protected StandaloneMesosServices(ActorSystem actorSystem, MesosArtifactServer artifactServer) { + super(actorSystem, artifactServer); + } @Override public MesosWorkerStore createMesosWorkerStore(Configuration configuration, Executor executor) { @@ -36,5 +43,6 @@ public MesosWorkerStore createMesosWorkerStore(Configuration configuration, Exec @Override public void close(boolean cleanup) throws Exception { + super.close(cleanup); } } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/ZooKeeperMesosServices.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/ZooKeeperMesosServices.java index 2883e4f1a884f..069cb833a3a05 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/ZooKeeperMesosServices.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/services/ZooKeeperMesosServices.java @@ -21,25 +21,31 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore; import org.apache.flink.mesos.runtime.clusterframework.store.ZooKeeperMesosWorkerStore; +import org.apache.flink.mesos.util.MesosArtifactServer; import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper; import org.apache.flink.runtime.zookeeper.ZooKeeperSharedCount; import org.apache.flink.runtime.zookeeper.ZooKeeperSharedValue; import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore; import org.apache.flink.runtime.zookeeper.ZooKeeperUtilityFactory; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; +import akka.actor.ActorSystem; + import java.util.concurrent.Executor; /** * {@link MesosServices} implementation for the ZooKeeper high availability based mode. */ -public class ZooKeeperMesosServices implements MesosServices { +public class ZooKeeperMesosServices extends AbstractMesosServices { // Factory to create ZooKeeper utility classes private final ZooKeeperUtilityFactory zooKeeperUtilityFactory; - public ZooKeeperMesosServices(ZooKeeperUtilityFactory zooKeeperUtilityFactory) { + public ZooKeeperMesosServices(ActorSystem actorSystem, MesosArtifactServer artifactServer, ZooKeeperUtilityFactory zooKeeperUtilityFactory) { + super(actorSystem, artifactServer); this.zooKeeperUtilityFactory = Preconditions.checkNotNull(zooKeeperUtilityFactory); } @@ -64,7 +70,23 @@ public MesosWorkerStore createMesosWorkerStore(Configuration configuration, Exec @Override public void close(boolean cleanup) throws Exception { - // this also closes the underlying CuratorFramework instance - zooKeeperUtilityFactory.close(cleanup); + Throwable exception = null; + + try { + // this also closes the underlying CuratorFramework instance + zooKeeperUtilityFactory.close(cleanup); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + + try { + super.close(cleanup); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + + if (exception != null) { + throw new FlinkException("Could not properly shut down the Mesos services.", exception); + } } } diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java index 8bfb4d120c15d..ff324865274e7 100644 --- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java +++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java @@ -251,6 +251,7 @@ public void initialize() { containeredParams, Collections.emptyList(), Collections.emptyList(), + "", Option.empty(), Option.empty()); 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 e81a2de3f1b47..4bbcb25e3373a 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.mesos.runtime.clusterframework.services.MesosServices; import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore; import org.apache.flink.mesos.scheduler.ConnectionMonitor; import org.apache.flink.mesos.scheduler.LaunchCoordinator; @@ -32,7 +33,7 @@ import org.apache.flink.mesos.scheduler.messages.Registered; import org.apache.flink.mesos.scheduler.messages.ResourceOffers; import org.apache.flink.mesos.scheduler.messages.StatusUpdate; -import org.apache.flink.mesos.util.MesosArtifactResolver; +import org.apache.flink.mesos.util.MesosArtifactServer; import org.apache.flink.mesos.util.MesosConfiguration; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.clusterframework.ApplicationStatus; @@ -159,17 +160,15 @@ public TestingMesosResourceManager( FatalErrorHandler fatalErrorHandler, // Mesos specifics - ActorSystem actorSystem, Configuration flinkConfig, + MesosServices mesosServices, MesosConfiguration mesosConfig, - MesosWorkerStore workerStore, MesosTaskManagerParameters taskManagerParameters, - ContainerSpecification taskManagerContainerSpec, - MesosArtifactResolver artifactResolver) { + ContainerSpecification taskManagerContainerSpec) { super(rpcService, resourceManagerEndpointId, resourceId, resourceManagerConfiguration, highAvailabilityServices, heartbeatServices, slotManager, metricRegistry, - jobLeaderIdService, fatalErrorHandler, actorSystem, flinkConfig, mesosConfig, workerStore, - taskManagerParameters, taskManagerContainerSpec, artifactResolver); + jobLeaderIdService, fatalErrorHandler, flinkConfig, mesosServices, mesosConfig, + taskManagerParameters, taskManagerContainerSpec); } @Override @@ -208,6 +207,7 @@ static class Context implements AutoCloseable { TestingRpcService rpcService; TestingFatalErrorHandler fatalErrorHandler; MockMesosResourceManagerRuntimeServices rmServices; + MockMesosServices mesosServices; // RM ResourceManagerConfiguration rmConfiguration; @@ -242,6 +242,7 @@ static class Context implements AutoCloseable { rpcService = new TestingRpcService(); fatalErrorHandler = new TestingFatalErrorHandler(); rmServices = new MockMesosResourceManagerRuntimeServices(); + mesosServices = new MockMesosServices(); // TaskExecutor templating ContainerSpecification containerSpecification = new ContainerSpecification(); @@ -249,7 +250,7 @@ static class Context implements AutoCloseable { new ContaineredTaskManagerParameters(1024, 768, 256, 4, new HashMap()); MesosTaskManagerParameters tmParams = new MesosTaskManagerParameters( 1.0, MesosTaskManagerParameters.ContainerType.MESOS, Option.empty(), containeredParams, - Collections.emptyList(), Collections.emptyList(), Option.empty(), + Collections.emptyList(), Collections.emptyList(), "", Option.empty(), Option.empty()); // resource manager @@ -270,13 +271,11 @@ static class Context implements AutoCloseable { rmServices.jobLeaderIdService, fatalErrorHandler, // Mesos specifics - system, flinkConfig, + mesosServices, rmServices.mesosConfig, - rmServices.workerStore, tmParams, - containerSpecification, - rmServices.artifactResolver + containerSpecification ); // TaskExecutors @@ -341,7 +340,7 @@ class MockMesosResourceManagerRuntimeServices extends MockResourceManagerRuntime public SchedulerDriver schedulerDriver; public MesosConfiguration mesosConfig; public MesosWorkerStore workerStore; - public MesosArtifactResolver artifactResolver; + public MesosArtifactServer artifactServer; MockMesosResourceManagerRuntimeServices() throws Exception { schedulerDriver = mock(SchedulerDriver.class); @@ -354,7 +353,28 @@ class MockMesosResourceManagerRuntimeServices extends MockResourceManagerRuntime workerStore = mock(MesosWorkerStore.class); when(workerStore.getFrameworkID()).thenReturn(Option.empty()); - artifactResolver = mock(MesosArtifactResolver.class); + artifactServer = mock(MesosArtifactServer.class); + } + } + + class MockMesosServices implements MesosServices { + @Override + public MesosWorkerStore createMesosWorkerStore(Configuration configuration, Executor executor) throws Exception { + return rmServices.workerStore; + } + + @Override + public ActorSystem getLocalActorSystem() { + return system; + } + + @Override + public MesosArtifactServer getArtifactServer() { + return rmServices.artifactServer; + } + + @Override + public void close(boolean cleanup) throws Exception { } } 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 old mode 100644 new mode 100755 index 2538f209c0ca0..1551933b4b116 --- 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 @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.clusterframework.BootstrapTools; @@ -46,6 +47,7 @@ import javax.annotation.concurrent.GuardedBy; +import java.io.IOException; import java.util.concurrent.Callable; import java.util.concurrent.Executor; @@ -92,6 +94,8 @@ protected void startCluster() { LOG.info("Starting {}.", getClass().getSimpleName()); try { + installDefaultFileSystem(configuration); + SecurityContext securityContext = installSecurityContext(configuration); securityContext.runSecured(new Callable() { @@ -115,6 +119,17 @@ public Void call() throws Exception { } } + protected void installDefaultFileSystem(Configuration configuration) throws Exception { + LOG.info("Install default filesystem."); + + try { + FileSystem.setDefaultScheme(configuration); + } catch (IOException e) { + throw new IOException("Error while setting the default " + + "filesystem scheme from configuration.", e); + } + } + protected SecurityContext installSecurityContext(Configuration configuration) throws Exception { LOG.info("Install security context."); @@ -184,9 +199,18 @@ protected MetricRegistry createMetricRegistry(Configuration configuration) { } protected void shutDown(boolean cleanupHaData) throws FlinkException { + LOG.info("Stopping {}.", getClass().getSimpleName()); + Throwable exception = null; synchronized (lock) { + + try { + stopClusterComponents(cleanupHaData); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + if (metricRegistry != null) { try { metricRegistry.shutdown(); @@ -244,6 +268,9 @@ protected abstract void startClusterComponents( HeartbeatServices heartbeatServices, MetricRegistry metricRegistry) throws Exception; + protected void stopClusterComponents(boolean cleanupHaData) throws Exception { + } + protected static ClusterConfiguration parseArguments(String[] args) { ParameterTool parameterTool = ParameterTool.fromArgs(args); 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 a7c6120dced53..e70f6c8ca7943 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 @@ -110,7 +110,7 @@ protected JobManagerRunner createJobManagerRunner( } @Override - protected void shutDown(boolean cleanupHaData) throws FlinkException { + protected void stopClusterComponents(boolean cleanupHaData) throws Exception { Throwable exception = null; if (jobManagerRunner != null) { @@ -129,14 +129,8 @@ protected void shutDown(boolean cleanupHaData) throws FlinkException { } } - try { - super.shutDown(cleanupHaData); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - if (exception != null) { - throw new FlinkException("Could not properly shut down the session cluster entry point.", exception); + throw new FlinkException("Could not properly shut down the job cluster entry point.", exception); } } 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 4013e8313c3d9..cea1688f13d3e 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 @@ -80,7 +80,7 @@ protected void startClusterComponents( } @Override - protected void shutDown(boolean cleanupHaData) throws FlinkException { + protected void stopClusterComponents(boolean cleanupHaData) throws Exception { Throwable exception = null; if (dispatcher != null) { @@ -99,12 +99,6 @@ protected void shutDown(boolean cleanupHaData) throws FlinkException { } } - try { - super.shutDown(cleanupHaData); - } catch (Throwable t) { - exception = ExceptionUtils.firstOrSuppressed(t, exception); - } - if (exception != null) { throw new FlinkException("Could not properly shut down the session cluster entry point.", exception); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/entrypoint/StreamingNoop.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/entrypoint/StreamingNoop.java new file mode 100644 index 0000000000000..cd88ae10deff1 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/entrypoint/StreamingNoop.java @@ -0,0 +1,60 @@ +/* + * 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.runtime.entrypoint; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.runtime.jobgraph.JobGraph; +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.FileMonitoringFunction; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.ObjectOutputStream; + +/** + * A program to generate a job graph for entrypoint testing purposes. + * + *

The dataflow is a simple streaming program that continuously monitors a (non-existent) directory. + * Note that the job graph doesn't depend on any user code; it uses in-built Flink classes only. + * + *

Program arguments: + * --output [graph file] (default: 'job.graph') + */ +public class StreamingNoop { + public static void main(String[] args) throws Exception { + ParameterTool params = ParameterTool.fromArgs(args); + + // define the dataflow + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(10, 1000)); + env.readFileStream("input/", 60000, FileMonitoringFunction.WatchType.ONLY_NEW_FILES) + .addSink(new DiscardingSink()); + + // generate a job graph + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + File jobGraphFile = new File(params.get("output", "job.graph")); + try (FileOutputStream output = new FileOutputStream(jobGraphFile); + ObjectOutputStream obOutput = new ObjectOutputStream(output)){ + obOutput.writeObject(jobGraph); + } + } +} From 40656c5dfdab7c0a1a7794dfe3a5f661f6156c6f Mon Sep 17 00:00:00 2001 From: "Wright, Eron" Date: Thu, 17 Aug 2017 18:22:55 -0700 Subject: [PATCH 030/129] [FLINK-7077] [mesos] Implement task release to support dynamic scaling - SlotManager: fix for idleness tracking (`markIdle` shouldn't reset `idleSince` on every call) - ResourceManager: change `stopWorker` method to use `ResourceID` - ResourceManager: schedule callbacks from `ResourceManagerActions` onto main thread - MesosResourceManager: implement `stopWorker` - MesosResourceManager: fix for message routing from child actors to RM This closes #4560. --- .../MesosFlinkResourceManager.java | 2 +- .../MesosResourceManager.java | 35 +++++++++++++--- .../apache/flink/mesos/scheduler/Tasks.scala | 8 ++-- .../MesosResourceManagerTest.java | 41 ++++++++++++++++--- .../flink/mesos/scheduler/TasksTest.scala | 2 +- .../resourcemanager/ResourceManager.java | 35 ++++++++++++++-- .../StandaloneResourceManager.java | 3 +- .../slotmanager/SlotManager.java | 3 ++ .../slotmanager/TaskManagerRegistration.java | 4 +- .../clusterframework/ResourceManagerTest.java | 2 +- .../TestingLeaderElectionService.java | 37 +++++++++++++---- .../flink/yarn/YarnResourceManager.java | 3 +- 12 files changed, 141 insertions(+), 34 deletions(-) diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java index 05d7e1f886f63..6335745004a18 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java @@ -192,7 +192,7 @@ protected ActorRef createConnectionMonitor() { protected ActorRef createTaskRouter() { return context().actorOf( - Tasks.createActorProps(Tasks.class, config, schedulerDriver, TaskMonitor.class), + Tasks.createActorProps(Tasks.class, self(), config, schedulerDriver, TaskMonitor.class), "tasks"); } 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 445010bc71a3d..9a2ad42a48cb5 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 @@ -49,7 +49,6 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManager; @@ -197,7 +196,7 @@ protected ActorRef createConnectionMonitor() { protected ActorRef createTaskMonitor(SchedulerDriver schedulerDriver) { return actorSystem.actorOf( - Tasks.createActorProps(Tasks.class, flinkConfig, schedulerDriver, TaskMonitor.class), + Tasks.createActorProps(Tasks.class, selfActor, flinkConfig, schedulerDriver, TaskMonitor.class), "tasks"); } @@ -422,8 +421,34 @@ public void startNewWorker(ResourceProfile resourceProfile) { } @Override - public void stopWorker(InstanceID instanceId) { - // TODO implement worker release + public void stopWorker(ResourceID resourceID) { + LOG.info("Stopping worker {}.", resourceID); + try { + + if (workersInLaunch.containsKey(resourceID)) { + // update persistent state of worker to Released + MesosWorkerStore.Worker worker = workersInLaunch.remove(resourceID); + worker = worker.releaseWorker(); + workerStore.putWorker(worker); + workersBeingReturned.put(extractResourceID(worker.taskID()), worker); + + taskMonitor.tell(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)), selfActor); + + if (worker.hostname().isDefined()) { + // tell the launch coordinator that the task is being unassigned from the host, for planning purposes + launchCoordinator.tell(new LaunchCoordinator.Unassign(worker.taskID(), worker.hostname().get()), selfActor); + } + } + else if (workersBeingReturned.containsKey(resourceID)) { + LOG.info("Ignoring request to stop worker {} because it is already being stopped.", resourceID); + } + else { + LOG.warn("Unrecognized worker {}.", resourceID); + } + } + catch (Exception e) { + onFatalErrorAsync(new ResourceManagerException("Unable to release a worker.", e)); + } } /** @@ -596,8 +621,6 @@ public void taskTerminated(TaskMonitor.TaskTerminated message) { assert(launched != null); LOG.info("Worker {} failed with status: {}, reason: {}, message: {}.", id, status.getState(), status.getReason(), status.getMessage()); - - // TODO : launch a replacement worker? } closeTaskManagerConnection(id, new Exception(status.getMessage())); diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala index 4f49c16134dce..54d1bd2179504 100644 --- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala +++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala @@ -34,6 +34,7 @@ import scala.collection.mutable.{Map => MutableMap} * Routes messages between the scheduler and individual task monitor actors. */ class Tasks( + manager: ActorRef, flinkConfig: Configuration, schedulerDriver: SchedulerDriver, taskMonitorCreator: (ActorRefFactory,TaskGoalState) => ActorRef) extends Actor { @@ -92,11 +93,11 @@ class Tasks( } case msg: Reconcile => - context.parent.forward(msg) + manager.forward(msg) case msg: TaskTerminated => taskMap.remove(msg.taskID) - context.parent.forward(msg) + manager.forward(msg) } private def createTask(task: TaskGoalState): ActorRef = { @@ -113,6 +114,7 @@ object Tasks { */ def createActorProps[T <: Tasks, M <: TaskMonitor]( actorClass: Class[T], + manager: ActorRef, flinkConfig: Configuration, schedulerDriver: SchedulerDriver, taskMonitorClass: Class[M]): Props = { @@ -122,6 +124,6 @@ object Tasks { factory.actorOf(props) } - Props.create(actorClass, flinkConfig, schedulerDriver, taskMonitorCreator) + Props.create(actorClass, manager, flinkConfig, schedulerDriver, taskMonitorCreator) } } 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 4bbcb25e3373a..cf0c913536475 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 @@ -78,6 +78,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; @@ -330,9 +331,9 @@ public Object answer(InvocationOnMock invocation) throws Throwable { when(slotManager.registerSlotRequest(any(SlotRequest.class))).thenReturn(true); } - public void grantLeadership() { + public void grantLeadership() throws Exception { rmLeaderSessionId = UUID.randomUUID(); - rmLeaderElectionService.isLeader(rmLeaderSessionId); + rmLeaderElectionService.isLeader(rmLeaderSessionId).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -454,7 +455,7 @@ public MesosWorkerStore.Worker allocateWorker(Protos.TaskID taskID, ResourceProf MesosWorkerStore.Worker expected = MesosWorkerStore.Worker.newWorker(taskID, resourceProfile); // drain the probe messages - verify(rmServices.workerStore).putWorker(expected); + verify(rmServices.workerStore, Mockito.timeout(timeout.toMilliseconds())).putWorker(expected); assertThat(resourceManager.workersInNew, hasEntry(extractResourceID(taskID), expected)); resourceManager.taskRouter.expectMsgClass(TaskMonitor.TaskGoalStateUpdated.class); resourceManager.launchCoordinator.expectMsgClass(LaunchCoordinator.Launch.class); @@ -531,7 +532,7 @@ public void testRequestNewWorkers() throws Exception { // verify that a new worker was persisted, the internal state was updated, the task router was notified, // and the launch coordinator was asked to launch a task MesosWorkerStore.Worker expected = MesosWorkerStore.Worker.newWorker(task1, resourceProfile1); - verify(rmServices.workerStore).putWorker(expected); + verify(rmServices.workerStore, Mockito.timeout(timeout.toMilliseconds())).putWorker(expected); assertThat(resourceManager.workersInNew, hasEntry(extractResourceID(task1), expected)); resourceManager.taskRouter.expectMsgClass(TaskMonitor.TaskGoalStateUpdated.class); resourceManager.launchCoordinator.expectMsgClass(LaunchCoordinator.Launch.class); @@ -617,7 +618,7 @@ public void testWorkerStarted() throws Exception { // send registration message CompletableFuture successfulFuture = resourceManager.registerTaskExecutor(rmServices.rmLeaderSessionId, task1Executor.address, task1Executor.resourceID, slotReport, timeout); - RegistrationResponse response = successfulFuture.get(5, TimeUnit.SECONDS); + RegistrationResponse response = successfulFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(response instanceof TaskExecutorRegistrationSuccess); // verify the internal state @@ -652,6 +653,36 @@ public void testWorkerFailed() throws Exception { }}; } + /** + * Test planned stop of a launched worker. + */ + @Test + public void testStopWorker() throws Exception { + new Context() {{ + // set the initial persistent state with a launched worker + MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newWorker(task1).launchWorker(slave1, slave1host); + when(rmServices.workerStore.getFrameworkID()).thenReturn(Option.apply(framework1)); + when(rmServices.workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched)); + startResourceManager(); + + // drain the assign message + resourceManager.launchCoordinator.expectMsgClass(LaunchCoordinator.Assign.class); + + // tell the RM to stop the worker + resourceManager.stopWorker(extractResourceID(task1)); + + // verify that the instance state was updated + MesosWorkerStore.Worker worker1Released = worker1launched.releaseWorker(); + verify(rmServices.workerStore).putWorker(worker1Released); + assertThat(resourceManager.workersInLaunch.entrySet(), empty()); + assertThat(resourceManager.workersBeingReturned, hasEntry(extractResourceID(task1), worker1Released)); + + // verify that the monitor was notified + resourceManager.taskRouter.expectMsgClass(TaskMonitor.TaskGoalStateUpdated.class); + resourceManager.launchCoordinator.expectMsgClass(LaunchCoordinator.Unassign.class); + }}; + } + /** * Test application shutdown handling. */ diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala index fcf2977af48ff..b3d9a5fddf169 100644 --- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala +++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala @@ -93,7 +93,7 @@ class TasksTest taskActorRef } TestActorRef[Tasks]( - Props(classOf[Tasks], config, schedulerDriver, taskActorCreator), + Props(classOf[Tasks], testActor, config, schedulerDriver, taskActorCreator), testActor, TestFSMUtils.randomName) } 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 e8ec0e006d21b..a9a9e5028f4bc 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 @@ -944,7 +944,12 @@ public void handleError(final Exception exception) { @VisibleForTesting public abstract void startNewWorker(ResourceProfile resourceProfile); - public abstract void stopWorker(InstanceID instanceId); + /** + * Deallocates a resource. + * + * @param resourceID The resource ID + */ + public abstract void stopWorker(ResourceID resourceID); /** * Callback when a worker was started. @@ -960,12 +965,36 @@ private class ResourceManagerActionsImpl implements ResourceManagerActions { @Override public void releaseResource(InstanceID instanceId) { - stopWorker(instanceId); + runAsync(new Runnable() { + @Override + public void run() { + ResourceID resourceID = null; + + for (Map.Entry> entry : taskExecutors.entrySet()) { + if (entry.getValue().getInstanceID().equals(instanceId)) { + resourceID = entry.getKey(); + break; + } + } + + if (resourceID != null) { + stopWorker(resourceID); + } + else { + log.warn("Ignoring request to release TaskManager with instance ID {} (not found).", instanceId); + } + } + }); } @Override public void allocateResource(ResourceProfile resourceProfile) throws ResourceManagerException { - startNewWorker(resourceProfile); + runAsync(new Runnable() { + @Override + public void run() { + startNewWorker(resourceProfile); + } + }); } @Override 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 a921a29183ba8..ac2c7453070e9 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 @@ -23,7 +23,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; @@ -76,7 +75,7 @@ public void startNewWorker(ResourceProfile resourceProfile) { } @Override - public void stopWorker(InstanceID instanceId) { + public void stopWorker(ResourceID resourceID) { } 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 3bda409f1a40a..52182860be16a 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 @@ -836,10 +836,13 @@ private void checkTaskManagerTimeouts() { while (taskManagerRegistrationIterator.hasNext()) { TaskManagerRegistration taskManagerRegistration = taskManagerRegistrationIterator.next().getValue(); + LOG.debug("Evaluating TaskManager {} for idleness.", taskManagerRegistration.getInstanceId()); if (anySlotUsed(taskManagerRegistration.getSlots())) { taskManagerRegistration.markUsed(); } else if (currentTime - taskManagerRegistration.getIdleSince() >= taskManagerTimeout.toMilliseconds()) { + LOG.info("Removing idle TaskManager {} from the SlotManager.", taskManagerRegistration.getInstanceId()); + taskManagerRegistrationIterator.remove(); internalUnregisterTaskManager(taskManagerRegistration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerRegistration.java index 7d3764c792c7d..f19f9bf15efff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskManagerRegistration.java @@ -68,7 +68,9 @@ public boolean isIdle() { } public void markIdle() { - idleSince = System.currentTimeMillis(); + if (!isIdle()) { + idleSince = System.currentTimeMillis(); + } } public void markUsed() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java index 9ad251b8a476e..737cede83d84e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java @@ -629,7 +629,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); - rmLeaderElectionService.isLeader(rmLeaderId); + rmLeaderElectionService.isLeader(rmLeaderId).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); // test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time CompletableFuture successfulFuture = rmGateway.registerJobManager( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java index d4560833e8ab5..d951db5de39a8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderElectionService.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.leaderelection; import java.util.UUID; +import java.util.concurrent.CompletableFuture; /** * Test {@link LeaderElectionService} implementation which directly forwards isLeader and notLeader @@ -28,43 +29,61 @@ public class TestingLeaderElectionService implements LeaderElectionService { private LeaderContender contender; private boolean hasLeadership = false; + private CompletableFuture confirmationFuture = null; + + /** + * Gets a future that completes when leadership is confirmed. + * + *

Note: the future is created upon calling {@link #isLeader(UUID)}. + */ + public synchronized CompletableFuture getConfirmationFuture() { + return confirmationFuture; + } @Override - public void start(LeaderContender contender) throws Exception { + public synchronized void start(LeaderContender contender) throws Exception { this.contender = contender; } @Override - public void stop() throws Exception { + public synchronized void stop() throws Exception { } @Override - public void confirmLeaderSessionID(UUID leaderSessionID) { - + public synchronized void confirmLeaderSessionID(UUID leaderSessionID) { + if (confirmationFuture != null) { + confirmationFuture.complete(leaderSessionID); + } } @Override - public boolean hasLeadership() { + public synchronized boolean hasLeadership() { return hasLeadership; } - public void isLeader(UUID leaderSessionID) { + public synchronized CompletableFuture isLeader(UUID leaderSessionID) { + if (confirmationFuture != null) { + confirmationFuture.cancel(false); + } + confirmationFuture = new CompletableFuture<>(); hasLeadership = true; contender.grantLeadership(leaderSessionID); + + return confirmationFuture; } - public void notLeader() { + public synchronized void notLeader() { hasLeadership = false; contender.revokeLeadership(); } - public void reset() { + public synchronized void reset() { contender = null; hasLeadership = false; } - public String getAddress() { + public synchronized String getAddress() { return contender.getAddress(); } } 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 fb1a1c3d1d1e1..c3398c44e0603 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 @@ -28,7 +28,6 @@ import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManager; @@ -228,7 +227,7 @@ public void startNewWorker(ResourceProfile resourceProfile) { } @Override - public void stopWorker(InstanceID instanceId) { + public void stopWorker(ResourceID resourceID) { // TODO: Implement to stop the worker } From bd70a00019f2c9fc01653d0229308635529aad73 Mon Sep 17 00:00:00 2001 From: zjureel Date: Fri, 28 Jul 2017 14:53:39 +0800 Subject: [PATCH 031/129] [FLINK-7269] Refactor passing of dynamic properties This closes #4415. --- .../flink/configuration/Configuration.java | 2 +- .../configuration/GlobalConfiguration.java | 60 +++++++++++-------- .../entrypoint/MesosEntrypointUtils.java | 22 ------- .../entrypoint/MesosJobClusterEntrypoint.java | 14 +++-- .../MesosSessionClusterEntrypoint.java | 14 +++-- .../entrypoint/MesosTaskExecutorRunner.java | 6 +- .../MesosApplicationMasterRunner.java | 3 +- .../MesosTaskManagerRunner.java | 3 +- 8 files changed, 64 insertions(+), 60 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java index d6f1decf3f69f..dfcd04fb97eae 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/Configuration.java @@ -79,7 +79,7 @@ public Configuration(Configuration other) { } // -------------------------------------------------------------------------------------------- - + /** * Returns the class associated with the given key as a string. * diff --git a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java index ea9f8bfc97e0c..4569ebe0acf0d 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/GlobalConfiguration.java @@ -28,6 +28,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + /** * Global configuration object for Flink. Similar to Java properties configuration * objects it includes key-value pairs which represent the framework's configuration. @@ -46,24 +48,6 @@ private GlobalConfiguration() {} // -------------------------------------------------------------------------------------------- - private static Configuration dynamicProperties = null; - - /** - * Set the process-wide dynamic properties to be merged with the loaded configuration. - */ - public static void setDynamicProperties(Configuration dynamicProperties) { - GlobalConfiguration.dynamicProperties = new Configuration(dynamicProperties); - } - - /** - * Get the dynamic properties. - */ - public static Configuration getDynamicProperties() { - return GlobalConfiguration.dynamicProperties; - } - - // -------------------------------------------------------------------------------------------- - /** * Loads the global configuration from the environment. Fails if an error occurs during loading. Returns an * empty configuration object if the environment variable is not set. In production this variable is set but @@ -76,18 +60,30 @@ public static Configuration loadConfiguration() { if (configDir == null) { return new Configuration(); } - return loadConfiguration(configDir); + return loadConfiguration(configDir, null); } /** * Loads the configuration files from the specified directory. *

* YAML files are supported as configuration files. - * + * * @param configDir * the directory which contains the configuration files */ public static Configuration loadConfiguration(final String configDir) { + return loadConfiguration(configDir, null); + } + + /** + * Loads the configuration files from the specified directory. If the dynamic properties + * configuration is not null, then it is added to the loaded configuration. + * + * @param configDir directory to load the configuration from + * @param dynamicProperties configuration file containing the dynamic properties. Null if none. + * @return The configuration loaded from the given configuration directory + */ + public static Configuration loadConfiguration(final String configDir, @Nullable final Configuration dynamicProperties) { if (configDir == null) { throw new IllegalArgumentException("Given configuration directory is null, cannot load configuration"); @@ -109,13 +105,29 @@ public static Configuration loadConfiguration(final String configDir) { "' (" + confDirFile.getAbsolutePath() + ") does not exist."); } - Configuration conf = loadYAMLResource(yamlConfigFile); + Configuration configuration = loadYAMLResource(yamlConfigFile); - if(dynamicProperties != null) { - conf.addAll(dynamicProperties); + if (dynamicProperties != null) { + configuration.addAll(dynamicProperties); + } + + return configuration; + } + + /** + * Loads the global configuration and adds the given dynamic properties + * configuration. + * + * @param dynamicProperties The given dynamic properties + * @return Returns the loaded global configuration with dynamic properties + */ + public static Configuration loadConfigurationWithDynamicProperties(Configuration dynamicProperties) { + final String configDir = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); + if (configDir == null) { + return new Configuration(dynamicProperties); } - return conf; + return loadConfiguration(configDir, dynamicProperties); } /** diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java index 0d81eadf71bac..368d62d8621f0 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosEntrypointUtils.java @@ -19,12 +19,10 @@ package org.apache.flink.mesos.entrypoint; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.mesos.configuration.MesosOptions; import org.apache.flink.mesos.runtime.clusterframework.MesosTaskManagerParameters; import org.apache.flink.mesos.util.MesosConfiguration; -import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.runtime.clusterframework.ContainerSpecification; import org.apache.flink.runtime.clusterframework.overlays.CompositeContainerOverlay; import org.apache.flink.runtime.clusterframework.overlays.FlinkDistributionOverlay; @@ -34,7 +32,6 @@ import org.apache.flink.runtime.clusterframework.overlays.Krb5ConfOverlay; import org.apache.flink.runtime.clusterframework.overlays.SSLStoreOverlay; -import org.apache.commons.cli.CommandLine; import org.apache.mesos.Protos; import org.slf4j.Logger; @@ -49,25 +46,6 @@ */ public class MesosEntrypointUtils { - /** - * Loads the global configuration and adds the dynamic properties parsed from - * the given command line. - * - * @param cmd command line to parse for dynamic properties - * @return Global configuration with dynamic properties set - * @deprecated replace once FLINK-7269 has been merged - */ - @Deprecated - public static Configuration loadConfiguration(CommandLine cmd) { - - // merge the dynamic properties from the command-line - Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); - GlobalConfiguration.setDynamicProperties(dynamicProperties); - Configuration config = GlobalConfiguration.loadConfiguration(); - - return config; - } - /** * Loads and validates the Mesos scheduler configuration. * @param flinkConfig the global configuration. diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java index 890c4a790c072..ba3b51db68d28 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosJobClusterEntrypoint.java @@ -43,6 +43,7 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -74,6 +75,8 @@ public class MesosJobClusterEntrypoint extends JobClusterEntrypoint { .addOption(BootstrapTools.newDynamicPropertiesOption()); } + private final Configuration dynamicProperties; + private MesosConfiguration schedulerConfiguration; private MesosServices mesosServices; @@ -82,8 +85,10 @@ public class MesosJobClusterEntrypoint extends JobClusterEntrypoint { private ContainerSpecification taskManagerContainerSpec; - public MesosJobClusterEntrypoint(Configuration config) { + public MesosJobClusterEntrypoint(Configuration config, Configuration dynamicProperties) { super(config); + + this.dynamicProperties = Preconditions.checkNotNull(dynamicProperties); } @Override @@ -100,7 +105,7 @@ protected void initializeServices(Configuration config) throws Exception { // TM configuration taskManagerParameters = MesosEntrypointUtils.createTmParameters(config, LOG); - taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, GlobalConfiguration.getDynamicProperties()); + taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, dynamicProperties); } @Override @@ -195,9 +200,10 @@ public static void main(String[] args) { return; } - Configuration configuration = MesosEntrypointUtils.loadConfiguration(cmd); + Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); + Configuration configuration = GlobalConfiguration.loadConfigurationWithDynamicProperties(dynamicProperties); - MesosJobClusterEntrypoint clusterEntrypoint = new MesosJobClusterEntrypoint(configuration); + MesosJobClusterEntrypoint clusterEntrypoint = new MesosJobClusterEntrypoint(configuration, dynamicProperties); clusterEntrypoint.startCluster(); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java index 67f5899c392bf..0ee2680a774f2 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosSessionClusterEntrypoint.java @@ -41,6 +41,7 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; @@ -64,6 +65,8 @@ public class MesosSessionClusterEntrypoint extends SessionClusterEntrypoint { .addOption(BootstrapTools.newDynamicPropertiesOption()); } + private final Configuration dynamicProperties; + private MesosConfiguration mesosConfig; private MesosServices mesosServices; @@ -72,8 +75,10 @@ public class MesosSessionClusterEntrypoint extends SessionClusterEntrypoint { private ContainerSpecification taskManagerContainerSpec; - public MesosSessionClusterEntrypoint(Configuration config) { + public MesosSessionClusterEntrypoint(Configuration config, Configuration dynamicProperties) { super(config); + + this.dynamicProperties = Preconditions.checkNotNull(dynamicProperties); } @Override @@ -90,7 +95,7 @@ protected void initializeServices(Configuration config) throws Exception { // TM configuration taskManagerParameters = MesosEntrypointUtils.createTmParameters(config, LOG); - taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, GlobalConfiguration.getDynamicProperties()); + taskManagerContainerSpec = MesosEntrypointUtils.createContainerSpec(config, dynamicProperties); } @Override @@ -169,9 +174,10 @@ public static void main(String[] args) { return; } - Configuration configuration = MesosEntrypointUtils.loadConfiguration(cmd); + Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); + Configuration configuration = GlobalConfiguration.loadConfigurationWithDynamicProperties(dynamicProperties); - MesosSessionClusterEntrypoint clusterEntrypoint = new MesosSessionClusterEntrypoint(configuration); + MesosSessionClusterEntrypoint clusterEntrypoint = new MesosSessionClusterEntrypoint(configuration, dynamicProperties); clusterEntrypoint.startCluster(); } diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java index c4343d27886d3..897e26d1b79f0 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/entrypoint/MesosTaskExecutorRunner.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.mesos.runtime.clusterframework.MesosConfigKeys; import org.apache.flink.runtime.clusterframework.BootstrapTools; @@ -71,7 +72,10 @@ public static void main(String[] args) throws Exception { final Configuration configuration; try { - configuration = MesosEntrypointUtils.loadConfiguration(cmd); + Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); + LOG.debug("Mesos dynamic properties: {}", dynamicProperties); + + configuration = GlobalConfiguration.loadConfigurationWithDynamicProperties(dynamicProperties); } catch (Throwable t) { LOG.error("Failed to load the TaskManager configuration and dynamic properties.", t); diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java index 3d16a664a2268..c0a68559501a9 100755 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java @@ -152,8 +152,7 @@ protected int run(final String[] args) { CommandLine cmd = parser.parse(ALL_OPTIONS, args); final Configuration dynamicProperties = BootstrapTools.parseDynamicProperties(cmd); - GlobalConfiguration.setDynamicProperties(dynamicProperties); - final Configuration config = GlobalConfiguration.loadConfiguration(); + final Configuration config = GlobalConfiguration.loadConfigurationWithDynamicProperties(dynamicProperties); // configure the default filesystem try { diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java index e1b0efa6a3818..4236a4341d915 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java @@ -73,10 +73,9 @@ public static void runTaskManager(String[] args, final Class Date: Mon, 21 Aug 2017 13:05:57 +0800 Subject: [PATCH 032/129] Fix bug that can not set offset to lastest --- .../eventhubs/FlinkEventHubConsumer.java | 33 ++++++++++++++++++- .../eventhubs/FlinkEventHubProducer.java | 4 +-- .../internals/EventhubClientWrapper.java | 2 +- 3 files changed, 35 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java index 109519aa079e7..ba4abd7bfdd5d 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -3,6 +3,7 @@ import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.configuration.Configuration; @@ -82,7 +83,7 @@ public FlinkEventHubConsumer(Properties eventhubsProps, KeyedDeserializationSche this.deserializer = deserializer; String userDefinedOffset = eventhubsProps.getProperty("eventhubs.auto.offset"); - if (userDefinedOffset != null && userDefinedOffset.toLowerCase() == "lastest"){ + if (userDefinedOffset != null && userDefinedOffset.toLowerCase().compareTo("lastest") == 0){ this.defaultEventhubInitOffset = PartitionReceiver.END_OF_STREAM; } else { @@ -266,6 +267,36 @@ public TypeInformation getProducedType() { return this.deserializer.getProducedType(); } + public FlinkEventHubConsumer assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks assigner) { + Preconditions.checkNotNull(assigner); + + if (this.periodicWatermarkAssigner != null) { + throw new IllegalStateException("A periodic watermark emitter has already been set."); + } + try { + ClosureCleaner.clean(assigner, true); + this.punctuatedWatermarkAssigner = new SerializedValue<>(assigner); + return this; + } catch (Exception e) { + throw new IllegalArgumentException("The given assigner is not serializable", e); + } + } + + public FlinkEventHubConsumer assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks assigner) { + Preconditions.checkNotNull(assigner); + + if (this.punctuatedWatermarkAssigner != null) { + throw new IllegalStateException("A punctuated watermark emitter has already been set."); + } + try { + ClosureCleaner.clean(assigner, true); + this.periodicWatermarkAssigner = new SerializedValue<>(assigner); + return this; + } catch (Exception e) { + throw new IllegalArgumentException("The given assigner is not serializable", e); + } + } + private List getAllEventhubPartitions() { List partitions = new ArrayList<>(); for (int i = 0; i < this.partitionCount; i++){ diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java index f6408f3b29018..61db348869c5f 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java @@ -58,12 +58,12 @@ public FlinkEventHubProducer(SerializationSchema serializationSchema, Prope @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { - cache.checkErr(); + return; } @Override public void initializeState(FunctionInitializationContext context) throws Exception { - + return; } @Override diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java index 277e958148ea3..9470482ccf5d0 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java @@ -95,7 +95,7 @@ else if (eventhubParams.containsKey("eventhubs.uri")){ String previousOffset = offset; - if (previousOffset != PartitionReceiver.START_OF_STREAM && previousOffset != null) { + if (previousOffset.compareTo(PartitionReceiver.START_OF_STREAM) != 0 && previousOffset != null) { offsetType = EventhubOffsetType.PreviousCheckpoint; currentOffset = previousOffset; From 39d94b5310b6dc6c348945ff8cc2980ddf17ea47 Mon Sep 17 00:00:00 2001 From: Joe Zhu Date: Thu, 15 Jun 2017 16:27:52 +0800 Subject: [PATCH 033/129] [FLINK-7474] [ Streaming Connectors] AzureEventhubs-connector, support read from and write to Azure eventhubs --- .../flink-connector-eventhubs/pom.xml | 141 +++++++ .../eventhubs/FlinkEventHubConsumer.java | 309 +++++++++++++++ .../eventhubs/FlinkEventHubProducer.java | 110 ++++++ .../eventhubs/internals/EventFetcher.java | 370 ++++++++++++++++++ .../internals/EventhubClientWrapper.java | 200 ++++++++++ .../internals/EventhubConsumerThread.java | 128 ++++++ .../internals/EventhubOffsetType.java | 12 + .../internals/EventhubPartition.java | 70 ++++ .../internals/EventhubPartitionState.java | 29 ++ ...bPartitionStateWithPeriodicWatermarks.java | 38 ++ ...artitionStateWithPunctuatedWatermarks.java | 48 +++ .../internals/EventhubProducerThread.java | 95 +++++ .../eventhubs/internals/Handover.java | 108 +++++ .../eventhubs/internals/ProducerCache.java | 124 ++++++ flink-connectors/pom.xml | 1 + .../flink-examples-streaming/pom.xml | 5 + .../examples/eventhub/ReadFromEventhub.java | 40 ++ .../examples/eventhub/WriteToEventhub.java | 53 +++ 18 files changed, 1881 insertions(+) create mode 100644 flink-connectors/flink-connector-eventhubs/pom.xml create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/ReadFromEventhub.java create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/WriteToEventhub.java diff --git a/flink-connectors/flink-connector-eventhubs/pom.xml b/flink-connectors/flink-connector-eventhubs/pom.xml new file mode 100644 index 0000000000000..70e3c075a7ad8 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/pom.xml @@ -0,0 +1,141 @@ + + + + flink-connectors + org.apache.flink + 1.4-SNAPSHOT + + 4.0.0 + + flink-connector-azureeventhubs_${scala.binary.version} + flink-connector-azureeventhubs + + jar + + + + + + com.microsoft.azure + azure-eventhubs + 0.14.0 + provided + + + + org.apache.flink + flink-connector-kafka-base_${scala.binary.version} + ${project.version} + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + provided + + true + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + test + test-jar + + + + org.apache.flink + flink-tests_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.apache.flink + flink-metrics-jmx + ${project.version} + test + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-test-sources + + test-jar-no-fork + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + -Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit + + + + + + + diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java new file mode 100644 index 0000000000000..ba4abd7bfdd5d --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -0,0 +1,309 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.ClosureCleaner; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.DefaultOperatorStateBackend; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.eventhubs.internals.EventFetcher; +import org.apache.flink.streaming.connectors.eventhubs.internals.EventhubPartition; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedValue; + +import com.microsoft.azure.eventhubs.PartitionReceiver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +/** + * Created by jozh on 5/22/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + * This class is used to create datastream from event hub + */ + +public class FlinkEventHubConsumer extends RichParallelSourceFunction implements + CheckpointedFunction, + ResultTypeQueryable, + CheckpointedRestoring> { + private static final long serialVersionUID = -3247976470793561346L; + protected static final Logger LOGGER = LoggerFactory.getLogger(FlinkEventHubConsumer.class); + protected static final String DEFAULTOFFSETSTATENAME = "flink.eventhub.offset"; + + protected final KeyedDeserializationSchema deserializer; + protected final Properties eventhubsProps; + protected final int partitionCount; + protected List> subscribedPartitions; + protected final String defaultEventhubInitOffset; + + private Map subscribedPartitionsToStartOffsets; + private SerializedValue> periodicWatermarkAssigner; + private SerializedValue> punctuatedWatermarkAssigner; + private transient ListState> offsetsStateForCheckpoint; + private transient volatile EventFetcher eventhubFetcher; + private transient volatile HashMap restoreToOffset; + private volatile boolean running = true; + + private Counter receivedCount; + + public FlinkEventHubConsumer(Properties eventhubsProps, DeserializationSchema deserializer){ + this(eventhubsProps, new KeyedDeserializationSchemaWrapper(deserializer)); + } + + public FlinkEventHubConsumer(Properties eventhubsProps, KeyedDeserializationSchema deserializer){ + Preconditions.checkNotNull(eventhubsProps); + Preconditions.checkNotNull(deserializer); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policyname")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policykey")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.namespace")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.name")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.partition.count")); + + this.eventhubsProps = eventhubsProps; + this.partitionCount = Integer.parseInt(eventhubsProps.getProperty("eventhubs.partition.count")); + this.deserializer = deserializer; + + String userDefinedOffset = eventhubsProps.getProperty("eventhubs.auto.offset"); + if (userDefinedOffset != null && userDefinedOffset.toLowerCase().compareTo("lastest") == 0){ + this.defaultEventhubInitOffset = PartitionReceiver.END_OF_STREAM; + } + else { + this.defaultEventhubInitOffset = PartitionReceiver.START_OF_STREAM; + } + + if (this.partitionCount <= 0){ + throw new IllegalArgumentException("eventhubs.partition.count must greater than 0"); + } + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + receivedCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("received_event_count"); + + List eventhubPartitions = this.getAllEventhubPartitions(); + this.subscribedPartitionsToStartOffsets = new HashMap<>(eventhubPartitions.size()); + + if (this.restoreToOffset != null){ + for (EventhubPartition partition : eventhubPartitions){ + if (this.restoreToOffset.containsKey(partition)){ + this.subscribedPartitionsToStartOffsets.put(partition, restoreToOffset.get(partition)); + } + } + + LOGGER.info("Consumer subtask {} will start reading {} partitions with offsets in restored state: {}", + getRuntimeContext().getIndexOfThisSubtask(), + this.subscribedPartitionsToStartOffsets.size(), + this.subscribedPartitionsToStartOffsets); + } + else { + //If there is no restored state. Then all partitions to read from start, the offset is "-1". In the + //future eventhub supports specify offset, we modify here + //We assign partition to each subTask in round robin mode + int numParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); + int indexofThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); + for (int i = 0; i < eventhubPartitions.size(); i++) { + if (i % numParallelSubtasks == indexofThisSubtask) { + this.subscribedPartitionsToStartOffsets.put(eventhubPartitions.get(i), defaultEventhubInitOffset); + } + } + + LOGGER.info("Consumer subtask {} will start reading {} partitions with offsets: {}", + getRuntimeContext().getIndexOfThisSubtask(), + this.subscribedPartitionsToStartOffsets.size(), + this.subscribedPartitionsToStartOffsets); + } + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + if (this.subscribedPartitionsToStartOffsets == null || this.subscribedPartitionsToStartOffsets.size() == 0){ + throw new Exception("The partitions were not set for the consumer"); + } + + StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext(); + + if (!this.subscribedPartitionsToStartOffsets.isEmpty()){ + final EventFetcher fetcher = new EventFetcher(sourceContext, + subscribedPartitionsToStartOffsets, + deserializer, + periodicWatermarkAssigner, + punctuatedWatermarkAssigner, + runtimeContext.getProcessingTimeService(), + runtimeContext.getExecutionConfig().getAutoWatermarkInterval(), + runtimeContext.getUserCodeClassLoader(), + runtimeContext.getTaskNameWithSubtasks(), + eventhubsProps, + false, + receivedCount); + + this.eventhubFetcher = fetcher; + if (!this.running){ + return; + } + + this.eventhubFetcher.runFetchLoop(); + } + else { + sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE)); + + final Object waitObj = new Object(); + while (this.running){ + try { + synchronized (waitObj){ + waitObj.wait(); + } + } + catch (InterruptedException ex){ + if (this.running){ + Thread.currentThread().interrupt(); + } + } + } + } + } + + @Override + public void close() throws Exception { + try { + this.cancel(); + } + finally { + super.close(); + } + } + + @Override + public void cancel() { + this.running = false; + + if (this.eventhubFetcher != null){ + this.eventhubFetcher.cancel(); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (!this.running){ + LOGGER.info("Consumer subtask {}: snapshotState() is called on the closed source", getRuntimeContext().getIndexOfThisSubtask()); + return; + } + + this.offsetsStateForCheckpoint.clear(); + final EventFetcher fetcher = this.eventhubFetcher; + if (fetcher == null){ + for (Map.Entry subscribedPartition : this.subscribedPartitionsToStartOffsets.entrySet()){ + this.offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue())); + } + } + else { + HashMap currentOffsets = fetcher.snapshotCurrentState(); + for (Map.Entry subscribedPartition : currentOffsets.entrySet()){ + this.offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue())); + } + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOGGER.info("Consumer subtask {}:Start init eventhub offset state", getRuntimeContext().getIndexOfThisSubtask()); + OperatorStateStore stateStore = context.getOperatorStateStore(); + /* this.offsetsStateForCheckpoint = stateStore + .getListState(new ListStateDescriptor>(DEFAULT_OFFSET_STATE_NAME, TypeInformation.of(new TypeHint>(){}))); +*/ + this.offsetsStateForCheckpoint = stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME); + if (context.isRestored()){ + if (this.restoreToOffset == null){ + this.restoreToOffset = new HashMap<>(); + for (Tuple2 offsetState : this.offsetsStateForCheckpoint.get()){ + this.restoreToOffset.put(offsetState.f0, offsetState.f1); + } + + LOGGER.info("Consumer subtask {}:Eventhub offset state is restored from checkpoint", getRuntimeContext().getIndexOfThisSubtask()); + } + else if (this.restoreToOffset.isEmpty()){ + this.restoreToOffset = null; + } + } + else { + LOGGER.info("Consumer subtask {}:No restore state for flink-eventhub-consumer", getRuntimeContext().getIndexOfThisSubtask()); + } + } + + @Override + public void restoreState(HashMap eventhubPartitionOffsets) throws Exception { + LOGGER.info("{} (taskIdx={}) restoring offsets from an older version.", + getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask()); + + this.restoreToOffset = eventhubPartitionOffsets; + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("{} (taskIdx={}) restored offsets from an older Flink version: {}", + getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), eventhubPartitionOffsets); + } + } + + @Override + public TypeInformation getProducedType() { + return this.deserializer.getProducedType(); + } + + public FlinkEventHubConsumer assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks assigner) { + Preconditions.checkNotNull(assigner); + + if (this.periodicWatermarkAssigner != null) { + throw new IllegalStateException("A periodic watermark emitter has already been set."); + } + try { + ClosureCleaner.clean(assigner, true); + this.punctuatedWatermarkAssigner = new SerializedValue<>(assigner); + return this; + } catch (Exception e) { + throw new IllegalArgumentException("The given assigner is not serializable", e); + } + } + + public FlinkEventHubConsumer assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks assigner) { + Preconditions.checkNotNull(assigner); + + if (this.punctuatedWatermarkAssigner != null) { + throw new IllegalStateException("A punctuated watermark emitter has already been set."); + } + try { + ClosureCleaner.clean(assigner, true); + this.periodicWatermarkAssigner = new SerializedValue<>(assigner); + return this; + } catch (Exception e) { + throw new IllegalArgumentException("The given assigner is not serializable", e); + } + } + + private List getAllEventhubPartitions() { + List partitions = new ArrayList<>(); + for (int i = 0; i < this.partitionCount; i++){ + partitions.add(new EventhubPartition(this.eventhubsProps, i)); + } + + LOGGER.info("Consumer subtask {}:Create {} eventhub partitions info", getRuntimeContext().getIndexOfThisSubtask(), this.partitionCount); + return partitions; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java new file mode 100644 index 0000000000000..61db348869c5f --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java @@ -0,0 +1,110 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.connectors.eventhubs.internals.EventhubProducerThread; +import org.apache.flink.streaming.connectors.eventhubs.internals.ProducerCache; +import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.flink.util.Preconditions; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +/** + * Created by jozh on 6/20/2017. + * Will support customize parttion in next version + */ +public class FlinkEventHubProducer extends RichSinkFunction implements CheckpointedFunction { + + private static final Logger logger = LoggerFactory.getLogger(FlinkEventHubProducer.class); + private static final long serialVersionUID = -7486455932880508035L; + private final SerializationSchema schema; + private final ProducerCache cache; + private final Properties eventhubsProps; + private EventhubProducerThread producerThread; + private Counter prepareSendCount; + private Counter commitSendCount; + + public FlinkEventHubProducer(SerializationSchema serializationSchema, Properties eventhubsProps){ + Preconditions.checkNotNull(serializationSchema); + Preconditions.checkNotNull(eventhubsProps); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policyname")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policykey")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.namespace")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.name")); + + this.schema = serializationSchema; + this.eventhubsProps = eventhubsProps; + + int capacity = eventhubsProps.getProperty("eventhubs.cache.capacity") == null + ? ProducerCache.DEFAULTCAPACITY : Integer.parseInt(eventhubsProps.getProperty("eventhubs.cache.capacity")); + + long timeout = eventhubsProps.getProperty("eventhubs.cache.timeout") == null + ? ProducerCache.DEFAULTTIMEOUTMILLISECOND : Long.parseLong(eventhubsProps.getProperty("eventhubs.cache.timeout")); + + this.cache = new ProducerCache(capacity, timeout); + + logger.info("Created eventhub producer for namespace: {}, name: {}", + eventhubsProps.getProperty("eventhubs.namespace"), + eventhubsProps.getProperty("eventhubs.name")); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + return; + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + return; + } + + @Override + public void invoke(OUT value) throws Exception { + cache.checkErr(); + EventData event = new EventData(this.schema.serialize(value)); + cache.put(event); + prepareSendCount.inc(); + logger.debug("Insert a event input output cache"); + cache.checkErr(); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + prepareSendCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("prepare_send_event_count"); + commitSendCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("commit_send_event_count"); + String threadName = getEventhubProducerName(); + + logger.info("Eventhub producer thread {} starting", threadName); + producerThread = new EventhubProducerThread( + logger, + threadName, + cache, + eventhubsProps, + commitSendCount); + producerThread.start(); + logger.info("Eventhub producer thread {} started", threadName); + cache.checkErr(); + } + + @Override + public void close() throws Exception { + super.close(); + logger.info("Eventhub producer thread close on demand"); + producerThread.shutdown(); + cache.close(); + cache.checkErr(); + } + + protected String getEventhubProducerName(){ + return "Eventhub producer " + getRuntimeContext().getTaskNameWithSubtasks(); + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java new file mode 100644 index 0000000000000..cae6327052930 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java @@ -0,0 +1,370 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.util.SerializedValue; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Created by jozh on 6/14/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector. + * A fetcher that fetches data from Eventhub via the EventhubUtil. + * Eventhub offset is stored at flink checkpoint backend + * @param The type of elements produced by the fetcher. + */ +public class EventFetcher { + protected static final int NO_TIMESTAMPS_WATERMARKS = 0; + protected static final int PERIODIC_WATERMARKS = 1; + protected static final int PUNCTUATED_WATERMARKS = 2; + private static final Logger logger = LoggerFactory.getLogger(EventFetcher.class); + private volatile boolean running = true; + + private final KeyedDeserializationSchema deserializer; + private final Handover handover; + private final Properties eventhubProps; + private final EventhubConsumerThread consumerThread; + private final String taskNameWithSubtasks; + + + protected final SourceFunction.SourceContext sourceContext; + protected final Object checkpointLock; + private final Map subscribedPartitionStates; + protected final int timestampWatermarkMode; + protected final boolean useMetrics; + private volatile long maxWatermarkSoFar = Long.MIN_VALUE; + private Counter receivedCount; + + public EventFetcher( + SourceFunction.SourceContext sourceContext, + Map assignedPartitionsWithInitialOffsets, + KeyedDeserializationSchema deserializer, + SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + ProcessingTimeService processTimerProvider, + long autoWatermarkInterval, + ClassLoader userCodeClassLoader, + String taskNameWithSubtasks, + Properties eventhubProps, + boolean useMetrics, + Counter receivedCount) throws Exception { + + this.sourceContext = checkNotNull(sourceContext); + this.deserializer = checkNotNull(deserializer); + this.eventhubProps = eventhubProps; + this.checkpointLock = sourceContext.getCheckpointLock(); + this.useMetrics = useMetrics; + this.receivedCount = receivedCount; + this.taskNameWithSubtasks = taskNameWithSubtasks; + this.timestampWatermarkMode = getTimestampWatermarkMode(watermarksPeriodic, watermarksPunctuated); + + this.subscribedPartitionStates = initializeSubscribedPartitionStates( + assignedPartitionsWithInitialOffsets, + timestampWatermarkMode, + watermarksPeriodic, watermarksPunctuated, + userCodeClassLoader); + + this.handover = new Handover(); + this.consumerThread = new EventhubConsumerThread(logger, + handover, + this.eventhubProps, + getFetcherName() + " for " + taskNameWithSubtasks, + this.subscribedPartitionStates.values().toArray(new EventhubPartitionState[this.subscribedPartitionStates.size()])); + + if (this.timestampWatermarkMode == PERIODIC_WATERMARKS) { + PeriodicWatermarkEmitter periodicEmitter = + new PeriodicWatermarkEmitter(this.subscribedPartitionStates, sourceContext, processTimerProvider, autoWatermarkInterval); + periodicEmitter.start(); + } + + } + + public HashMap snapshotCurrentState() { + // this method assumes that the checkpoint lock is held + logger.debug("snapshot current offset state for subtask {}", taskNameWithSubtasks); + assert Thread.holdsLock(checkpointLock); + + HashMap state = new HashMap<>(subscribedPartitionStates.size()); + for (Map.Entry partition : subscribedPartitionStates.entrySet()){ + state.put(partition.getKey(), partition.getValue().getOffset()); + } + + return state; + } + + public void runFetchLoop() throws Exception{ + try { + final Handover handover = this.handover; + consumerThread.start(); + logger.info("Eventhub consumer thread started for substask {}", taskNameWithSubtasks); + + logger.info("Start fetcher loop to get data from eventhub and emit to flink for subtask {}", taskNameWithSubtasks); + while (running){ + final Tuple2> eventsTuple = handover.pollNext(); + for (EventData event : eventsTuple.f1){ + final T value = deserializer.deserialize(null, + event.getBytes(), + event.getSystemProperties().getPartitionKey(), + eventsTuple.f0.getParitionId(), + event.getSystemProperties().getSequenceNumber()); + + if (deserializer.isEndOfStream(value)){ + running = false; + break; + } + emitRecord(value, subscribedPartitionStates.get(eventsTuple.f0), event.getSystemProperties().getOffset()); + receivedCount.inc(); + } + } + } + finally { + logger.warn("Stopping eventhub consumer thread of subtask {}, because something wrong when deserializing received event " + , taskNameWithSubtasks); + consumerThread.shutdown(); + } + + try { + consumerThread.join(); + logger.warn("Waiting eventhub consumer thread of subtask {} stopped", taskNameWithSubtasks); + } + catch (InterruptedException ex){ + Thread.currentThread().interrupt(); + } + + logger.info("EventFetcher of subtask {} stopped", taskNameWithSubtasks); + } + + public void cancel(){ + logger.info("EventFetcher of subtask {} canceled on demand", taskNameWithSubtasks); + running = false; + handover.close(); + consumerThread.shutdown(); + } + + protected void emitRecord(T record, EventhubPartitionState partitionState, String offset) throws Exception{ + if (record == null){ + synchronized (this.checkpointLock){ + partitionState.setOffset(offset); + } + return; + } + + if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS){ + synchronized (this.checkpointLock){ + sourceContext.collect(record); + partitionState.setOffset(offset); + } + } + else if (timestampWatermarkMode == PERIODIC_WATERMARKS){ + emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, Long.MIN_VALUE); + } + else { + emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, Long.MIN_VALUE); + } + } + + protected void emitRecordWithTimestampAndPunctuatedWatermark( + T record, + EventhubPartitionState partitionState, + String offset, + long eventTimestamp) { + + final EventhubPartitionStateWithPeriodicWatermarks withWatermarksState = + (EventhubPartitionStateWithPeriodicWatermarks) partitionState; + + final long timestamp; + synchronized (withWatermarksState) { + timestamp = withWatermarksState.getTimestampForRecord(record, eventTimestamp); + } + + synchronized (checkpointLock) { + sourceContext.collectWithTimestamp(record, timestamp); + partitionState.setOffset(offset); + } + } + + protected void emitRecordWithTimestampAndPeriodicWatermark( + T record, + EventhubPartitionState partitionState, + String offset, + long eventTimestamp) { + + final EventhubPartitionStateWithPunctuatedWatermarks withWatermarksState = + (EventhubPartitionStateWithPunctuatedWatermarks) partitionState; + + final long timestamp = withWatermarksState.getTimestampForRecord(record, eventTimestamp); + final Watermark newWatermark = withWatermarksState.checkAndGetNewWatermark(record, timestamp); + + synchronized (checkpointLock) { + sourceContext.collectWithTimestamp(record, timestamp); + partitionState.setOffset(offset); + } + + if (newWatermark != null) { + updateMinPunctuatedWatermark(newWatermark); + } + } + + protected String getFetcherName() { + return "Eventhubs Fetcher"; + } + + private int getTimestampWatermarkMode(SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated) + throws IllegalArgumentException { + if (watermarksPeriodic == null){ + if (watermarksPunctuated == null){ + return NO_TIMESTAMPS_WATERMARKS; + } + else { + return PUNCTUATED_WATERMARKS; + } + } + else { + if (watermarksPunctuated == null){ + return PERIODIC_WATERMARKS; + } + else { + throw new IllegalArgumentException("Cannot have both periodic and punctuated watermarks"); + } + } + } + + private Map initializeSubscribedPartitionStates( + Map assignedPartitionsWithInitialOffsets, + int timestampWatermarkMode, + SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + ClassLoader userCodeClassLoader) throws IOException, ClassNotFoundException { + + if (timestampWatermarkMode != NO_TIMESTAMPS_WATERMARKS + && timestampWatermarkMode != PERIODIC_WATERMARKS + && timestampWatermarkMode != PUNCTUATED_WATERMARKS) { + throw new RuntimeException(); + } + + Map partitionsState = new HashMap<>(assignedPartitionsWithInitialOffsets.size()); + for (Map.Entry partition : assignedPartitionsWithInitialOffsets.entrySet()){ + switch (timestampWatermarkMode){ + case NO_TIMESTAMPS_WATERMARKS:{ + partitionsState.put(partition.getKey(), new EventhubPartitionState(partition.getKey(), partition.getValue())); + logger.info("NO_TIMESTAMPS_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue()); + break; + } + + case PERIODIC_WATERMARKS:{ + AssignerWithPeriodicWatermarks assignerInstance = + watermarksPeriodic.deserializeValue(userCodeClassLoader); + partitionsState.put(partition.getKey(), + new EventhubPartitionStateWithPeriodicWatermarks(partition.getKey(), partition.getValue(), assignerInstance)); + logger.info("PERIODIC_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue()); + break; + } + + case PUNCTUATED_WATERMARKS: { + AssignerWithPunctuatedWatermarks assignerInstance = + watermarksPunctuated.deserializeValue(userCodeClassLoader); + partitionsState.put(partition.getKey(), + new EventhubPartitionStateWithPunctuatedWatermarks(partition.getKey(), partition.getValue(), assignerInstance)); + logger.info("PUNCTUATED_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue()); + break; + } + } + } + return partitionsState; + } + + private void updateMinPunctuatedWatermark(Watermark nextWatermark) { + if (nextWatermark.getTimestamp() > maxWatermarkSoFar) { + long newMin = Long.MAX_VALUE; + + for (Map.Entry partition : subscribedPartitionStates.entrySet()){ + final EventhubPartitionStateWithPunctuatedWatermarks withWatermarksState = + (EventhubPartitionStateWithPunctuatedWatermarks) partition.getValue(); + + newMin = Math.min(newMin, withWatermarksState.getCurrentPartitionWatermark()); + } + + // double-check locking pattern + if (newMin > maxWatermarkSoFar) { + synchronized (checkpointLock) { + if (newMin > maxWatermarkSoFar) { + maxWatermarkSoFar = newMin; + sourceContext.emitWatermark(new Watermark(newMin)); + } + } + } + } + } + + private static class PeriodicWatermarkEmitter implements ProcessingTimeCallback { + + private final Map allPartitions; + + private final SourceFunction.SourceContext emitter; + + private final ProcessingTimeService timerService; + + private final long interval; + + private long lastWatermarkTimestamp; + + //------------------------------------------------- + + PeriodicWatermarkEmitter( + Map allPartitions, + SourceFunction.SourceContext emitter, + ProcessingTimeService timerService, + long autoWatermarkInterval) { + this.allPartitions = checkNotNull(allPartitions); + this.emitter = checkNotNull(emitter); + this.timerService = checkNotNull(timerService); + this.interval = autoWatermarkInterval; + this.lastWatermarkTimestamp = Long.MIN_VALUE; + } + + public void start() { + timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this); + } + + @Override + public void onProcessingTime(long timestamp) throws Exception { + + long minAcrossAll = Long.MAX_VALUE; + for (Map.Entry partition : allPartitions.entrySet()){ + final long curr; + EventhubPartitionStateWithPeriodicWatermarks state = + (EventhubPartitionStateWithPeriodicWatermarks) partition.getValue(); + + synchronized (state) { + curr = state.getCurrentWatermarkTimestamp(); + } + + minAcrossAll = Math.min(minAcrossAll, curr); + } + + if (minAcrossAll > lastWatermarkTimestamp) { + lastWatermarkTimestamp = minAcrossAll; + emitter.emitWatermark(new Watermark(minAcrossAll)); + } + + timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this); + } + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java new file mode 100644 index 0000000000000..9470482ccf5d0 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java @@ -0,0 +1,200 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.util.Preconditions; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.ServiceBusException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.ExecutionException; + +/** + * Created by jozh on 6/14/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ +public class EventhubClientWrapper implements Serializable { + private static final long serialVersionUID = -5319150387753930840L; + private static final Logger logger = LoggerFactory.getLogger(EventhubClientWrapper.class); + private EventHubClient eventHubClient; + private PartitionReceiver eventhubReceiver; + private ConnectionStringBuilder connectionString; + private String consumerGroup; + private Long receiverEpoch; + + private Duration receiverTimeout; + private EventhubOffsetType offsetType; + private String currentOffset; + private String partitionId; + + private final int minPrefetchCount = 10; + private int maxPrefetchCount = 999; + private int maxEventRate = 0; + private final Long defaultReceiverEpoch = -1L; + private final String defaultReceiverTimeout = "60000"; + + public void createReveiver(Properties eventhubParams, String partitionId) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10")); + this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM); + } + + public void createReveiver(Properties eventhubParams, String partitionId, String offset) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10")); + this.createReveiver(eventhubParams, partitionId, maxEventRate, offset); + } + + public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM); + } + + /*Will not implement a standalone offset store here, will leverage flink state to save the offset of eventhub*/ + public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate, String offset) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + if (eventhubParams.containsKey("eventhubs.uri") && eventhubParams.containsKey("eventhubs.namespace")) { + throw new IllegalArgumentException("Eventhubs URI and namespace cannot both be specified at the same time."); + } + + if (eventhubParams.containsKey("eventhubs.namespace")){ + this.connectionString = new ConnectionStringBuilder( + eventhubParams.getProperty("eventhubs.namespace"), + eventhubParams.getProperty("eventhubs.name"), + eventhubParams.getProperty("eventhubs.policyname"), + eventhubParams.getProperty("eventhubs.policykey")); + } + else if (eventhubParams.containsKey("eventhubs.uri")){ + this.connectionString = new ConnectionStringBuilder(new URI( + eventhubParams.getProperty("eventhubs.uri")), + eventhubParams.getProperty("eventhubs.name"), + eventhubParams.getProperty("eventhubs.policyname"), + eventhubParams.getProperty("eventhubs.policykey")); + } + else { + throw new IllegalArgumentException("Either Eventhubs URI or namespace nust be specified."); + } + + this.partitionId = Preconditions.checkNotNull(partitionId, "partitionId is no valid, cannot be null or empty"); + this.consumerGroup = eventhubParams.getProperty("eventhubs.consumergroup", EventHubClient.DEFAULT_CONSUMER_GROUP_NAME); + this.receiverEpoch = Long.parseLong(eventhubParams.getProperty("eventhubs.epoch", defaultReceiverEpoch.toString())); + this.receiverTimeout = Duration.ofMillis(Long.parseLong(eventhubParams.getProperty("eventhubs.receiver.timeout", defaultReceiverTimeout))); + this.offsetType = EventhubOffsetType.None; + this.currentOffset = PartitionReceiver.START_OF_STREAM; + + String previousOffset = offset; + + if (previousOffset.compareTo(PartitionReceiver.START_OF_STREAM) != 0 && previousOffset != null) { + + offsetType = EventhubOffsetType.PreviousCheckpoint; + currentOffset = previousOffset; + + } else if (eventhubParams.containsKey("eventhubs.filter.offset")) { + + offsetType = EventhubOffsetType.InputByteOffset; + currentOffset = eventhubParams.getProperty("eventhubs.filter.offset"); + + } else if (eventhubParams.containsKey("eventhubs.filter.enqueuetime")) { + + offsetType = EventhubOffsetType.InputTimeOffset; + currentOffset = eventhubParams.getProperty("eventhubs.filter.enqueuetime"); + } + + this.maxEventRate = maxEventRate; + + if (maxEventRate > 0 && maxEventRate < minPrefetchCount) { + maxPrefetchCount = minPrefetchCount; + } + else if (maxEventRate >= minPrefetchCount && maxEventRate < maxPrefetchCount) { + maxPrefetchCount = maxEventRate + 1; + } + else { + maxEventRate = maxPrefetchCount - 1; + } + + this.createReceiverInternal(); + } + + public Iterable receive () throws ExecutionException, InterruptedException { + return this.eventhubReceiver.receive(maxEventRate).get(); + } + + public void close(){ + logger.info("Close eventhub client on demand of partition {}", this.partitionId); + if (this.eventhubReceiver != null){ + try { + this.eventhubReceiver.closeSync(); + } + catch (ServiceBusException ex){ + logger.error("Close eventhub client of partition {} failed, reason: {}", this.partitionId, ex.getMessage()); + } + } + } + + private void createReceiverInternal() throws IOException, ServiceBusException{ + this.eventHubClient = EventHubClient.createFromConnectionStringSync(this.connectionString.toString()); + + switch (this.offsetType){ + case None: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case PreviousCheckpoint: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case InputByteOffset: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case InputTimeOffset: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset)), receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset))); + } + break; + } + } + + this.eventhubReceiver.setPrefetchCount(maxPrefetchCount); + this.eventhubReceiver.setReceiveTimeout(this.receiverTimeout); + logger.info("Successfully create eventhub receiver for partition {}, max_event_rate {}, max_prefetch_rate {}, receive_timeout {}, offset {}, ", + this.partitionId, + this.maxEventRate, + this.maxPrefetchCount, + this.receiverTimeout, + this.currentOffset); + } + + public Duration getReceiverTimeout() { + return receiverTimeout; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java new file mode 100644 index 0000000000000..c2c37bbb58c9b --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java @@ -0,0 +1,128 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.api.java.tuple.Tuple2; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * Created by jozh on 5/24/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + * Cause eventhub client can only access one partition at one time, so here we should have multiple eventhub clients + * In this worker thread, it will receive event from each partition in round robin mode, any partition failed to retrive + * events will lead thread exception, and leverage flink HA framework to start from begining again + */ +public class EventhubConsumerThread extends Thread { + private final Logger logger; + private final Handover handover; + private final Properties eventhubProps; + private final EventhubPartitionState[] subscribedPartitionStates; + private final Map clients; + private volatile boolean running; + + public EventhubConsumerThread( + Logger logger, + Handover handover, + Properties eventhubProps, + String threadName, + EventhubPartitionState[] subscribedPartitionStates) throws Exception{ + + super(threadName); + setDaemon(true); + + this.logger = logger; + this.handover = handover; + this.eventhubProps = eventhubProps; + this.subscribedPartitionStates = subscribedPartitionStates; + this.running = true; + + this.clients = new HashMap<>(this.subscribedPartitionStates.length); + for (int i = 0; i < this.subscribedPartitionStates.length; i++){ + EventhubClientWrapper client = new EventhubClientWrapper(); + this.clients.put(this.subscribedPartitionStates[i], client); + } + } + + public void shutdown(){ + logger.info("Shutdown eventhub consumer thread {} on demand", this.getName()); + running = false; + handover.wakeupProducer(); + } + + @Override + public void run() { + if (!running){ + logger.info("Eventhub consumer thread is set to STOP, thread {} exit", this.getName()); + return; + } + + try { + logger.info("Starting create {} eventhub clients on {}", this.subscribedPartitionStates.length, this.getName()); + for (Map.Entry client : clients.entrySet()){ + EventhubPartitionState state = client.getKey(); + client.getValue().createReveiver(this.eventhubProps, Integer.toString(state.getPartition().getParitionId()), state.getOffset()); + } + } + catch (Throwable t){ + logger.error("Create eventhub client of {}, error: {}", this.getName(), t); + handover.reportError(t); + clearReceiveClients(); + return; + } + + try { + int currentClientIndex = 0; + while (running){ + EventhubPartitionState partitionState = subscribedPartitionStates[currentClientIndex]; + EventhubClientWrapper client = clients.get(partitionState); + Iterable events = client.receive(); + if (events != null){ + handover.produce(Tuple2.of(partitionState.getPartition(), events)); + logger.debug("Received event from {} on {}", partitionState.getPartition().toString(), this.getName()); + } + else { + logger.warn("Receive events from {} timeout, timeout set to {}, thread {}", + partitionState.getPartition().toString(), + client.getReceiverTimeout(), + this.getName()); + } + + currentClientIndex++; + currentClientIndex = currentClientIndex % subscribedPartitionStates.length; + } + } + catch (Throwable t){ + logger.error("Receving events error, {}", t); + handover.reportError(t); + } + finally { + logger.info("Exit from eventhub consumer thread, {}", this.getName()); + handover.close(); + clearReceiveClients(); + } + + logger.info("EventhubConsumerThread {} quit", this.getName()); + } + + private void clearReceiveClients(){ + if (clients == null){ + return; + } + + for (Map.Entry client : clients.entrySet()){ + try { + client.getValue().close(); + logger.info("Eventhub client for partition {} closed", client.getKey().getPartition().getParitionId()); + } + catch (Throwable t){ + logger.warn("Error while close eventhub client for partition {}, error is {}", + client.getKey().getPartition().getParitionId(), + t.getMessage()); + } + } + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java new file mode 100644 index 0000000000000..950efde56b824 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java @@ -0,0 +1,12 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +/** + * Created by jozh on 5/22/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ +public enum EventhubOffsetType { + None, + PreviousCheckpoint, + InputByteOffset, + InputTimeOffset +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java new file mode 100644 index 0000000000000..f8ac4d6c5f7c8 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java @@ -0,0 +1,70 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; + +import java.io.Serializable; +import java.util.Properties; + +/** + * Created by jozh on 5/23/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ + +public class EventhubPartition implements Serializable { + private static final long serialVersionUID = 134878919919793479L; + private final int cachedHash; + private final String policyName; + private final String policyKey; + private final String namespace; + private final String name; + + public int getParitionId() { + return paritionId; + } + + public String getPartitionName(){ + return namespace + "-" + name; + } + + private final int paritionId; + + public EventhubPartition(Properties props, int parition){ + this(props.getProperty("eventhubs.policyname"), + props.getProperty("eventhubs.policykey"), + props.getProperty("eventhubs.namespace"), + props.getProperty("eventhubs.name"), + parition); + } + + public EventhubPartition(String policyName, String policyKey, String namespace, String name, int paritionId){ + Preconditions.checkArgument(paritionId >= 0); + + this.policyName = Preconditions.checkNotNull(policyName); + this.policyKey = Preconditions.checkNotNull(policyKey); + this.name = Preconditions.checkNotNull(name); + this.namespace = Preconditions.checkNotNull(namespace); + this.paritionId = paritionId; + this.cachedHash = 31 * (this.namespace + this.name).hashCode() + paritionId; + } + + @Override + public String toString() { + return "EventhubPartition, namespace: " + this.namespace + + " name: " + this.name + + " partition: " + this.paritionId; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof EventhubPartition){ + return this.hashCode() == ((EventhubPartition) obj).hashCode(); + } + + return false; + } + + @Override + public int hashCode() { + return this.cachedHash; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java new file mode 100644 index 0000000000000..3403ccac0d69e --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java @@ -0,0 +1,29 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +/** + * Created by jozh on 5/23/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ + +public class EventhubPartitionState { + private final EventhubPartition partition; + private volatile String offset; + + public EventhubPartitionState(EventhubPartition partition, String offset){ + this.partition = partition; + this.offset = offset; + } + + public final String getOffset() { + return this.offset; + } + + public final void setOffset(String offset) { + this.offset = offset; + } + + public EventhubPartition getPartition() { + return this.partition; + } +} + diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java new file mode 100644 index 0000000000000..2882703761baa --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java @@ -0,0 +1,38 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; + +/** + * Created by jozh on 6/16/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ + +public class EventhubPartitionStateWithPeriodicWatermarks extends EventhubPartitionState { + private final AssignerWithPeriodicWatermarks timestampsAndWatermarks; + private long partitionWatermark; + + public EventhubPartitionStateWithPeriodicWatermarks(EventhubPartition key, String value, AssignerWithPeriodicWatermarks timestampsAndWatermarks) { + super(key, value); + this.timestampsAndWatermarks = timestampsAndWatermarks; + this.partitionWatermark = Long.MIN_VALUE; + } + + public long getTimestampForRecord(T record, long kafkaEventTimestamp) { + return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp); + } + + public long getCurrentWatermarkTimestamp() { + Watermark wm = timestampsAndWatermarks.getCurrentWatermark(); + if (wm != null) { + partitionWatermark = Math.max(partitionWatermark, wm.getTimestamp()); + } + return partitionWatermark; + } + + @Override + public String toString() { + return "EventhubPartitionStateWithPeriodicWatermarks: partition=" + getPartition() + + ", offset=" + getOffset() + ", watermark=" + partitionWatermark; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java new file mode 100644 index 0000000000000..d68f8a00a077a --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java @@ -0,0 +1,48 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; + +import javax.annotation.Nullable; + +/** + * Created by jozh on 6/16/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ + +public class EventhubPartitionStateWithPunctuatedWatermarks extends EventhubPartitionState { + private final AssignerWithPunctuatedWatermarks timestampsAndWatermarks; + private long partitionWatermark; + + public EventhubPartitionStateWithPunctuatedWatermarks(EventhubPartition key, String value, AssignerWithPunctuatedWatermarks timestampsAndWatermarks) { + super(key, value); + this.timestampsAndWatermarks = timestampsAndWatermarks; + this.partitionWatermark = Long.MIN_VALUE; + } + + public long getTimestampForRecord(T record, long kafkaEventTimestamp) { + return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp); + } + + @Nullable + public Watermark checkAndGetNewWatermark(T record, long timestamp) { + Watermark mark = timestampsAndWatermarks.checkAndGetNextWatermark(record, timestamp); + if (mark != null && mark.getTimestamp() > partitionWatermark) { + partitionWatermark = mark.getTimestamp(); + return mark; + } + else { + return null; + } + } + + public long getCurrentPartitionWatermark() { + return partitionWatermark; + } + + @Override + public String toString() { + return "EventhubPartitionStateWithPunctuatedWatermarks: partition=" + getPartition() + + ", offset=" + getOffset() + ", watermark=" + partitionWatermark; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java new file mode 100644 index 0000000000000..1cc1fd3dd2626 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java @@ -0,0 +1,95 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.metrics.Counter; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.ServiceBusException; +import org.slf4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Properties; +/** + * Created by jozh on 6/20/2017. + */ + +public class EventhubProducerThread extends Thread { + private final Logger logger; + private final ProducerCache producerCache; + private final Properties eventhubProps; + private final EventHubClient producer; + private volatile boolean running; + private Counter commitSendCount; + + public EventhubProducerThread( + Logger logger, + String threadName, + ProducerCache producerCache, + Properties eventhubProps, + Counter commitSendCount) throws IOException, ServiceBusException{ + + super(threadName); + setDaemon(true); + + this.logger = logger; + this.producerCache = producerCache; + this.eventhubProps = eventhubProps; + this.commitSendCount = commitSendCount; + + ConnectionStringBuilder connectionStringBuilder = new ConnectionStringBuilder( + eventhubProps.getProperty("eventhubs.namespace"), + eventhubProps.getProperty("eventhubs.name"), + eventhubProps.getProperty("eventhubs.policyname"), + eventhubProps.getProperty("eventhubs.policykey")); + this.producer = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); + this.running = true; + } + + public void shutdown(){ + logger.info("Shutdown eventhub producer thread {} on demand", this.getName()); + running = false; + } + + @Override + public void run() { + if (!running){ + logger.info("Eventhub producer thread is set to STOP, thread {} exit", this.getName()); + return; + } + + try { + logger.info("Eventhub producer thread {} started", this.getName()); + while (running){ + final ArrayList events = producerCache.pollNextBatch(); + if (events != null && events.size() > 0){ + producer.sendSync(events); + commitSendCount.inc(events.size()); + logger.info("Eventhub producer thread send {} events success", events.size()); + } + else { + logger.debug("Eventhub producer thread received a null eventdata from producer cache"); + } + } + } + catch (Throwable t){ + logger.error("Sending events error, {}", t.toString()); + producerCache.reportError(t); + } + finally { + logger.info("Exit from eventhub producer thread, {}", this.getName()); + if (producer != null){ + try { + producer.closeSync(); + } + catch (Exception ex) { + logger.error("Close eventhubclient {} error {}", eventhubProps.getProperty("eventhubs.name"), ex.getMessage()); + producerCache.reportError(ex); + } + } + } + + logger.info("EventhubProducerThread {} quit", this.getName()); + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java new file mode 100644 index 0000000000000..cc6098f078c3c --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java @@ -0,0 +1,108 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.ExceptionUtils; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.ThreadSafe; + +import java.io.Closeable; +import java.util.concurrent.ConcurrentLinkedQueue; + + +/** + * Created by jozh on 5/23/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ +@ThreadSafe +public final class Handover implements Closeable { + private static final int MAX_EVENTS_BLOCK_IN_QUEUE = 1000; + private static final Logger logger = LoggerFactory.getLogger(Handover.class); + private ConcurrentLinkedQueue>> eventQueue = new ConcurrentLinkedQueue(); + + private volatile boolean allProducerWakeup = true; + + private Throwable error; + + public Tuple2> pollNext() throws Exception{ + logger.debug("###Begin to poll data from event cache queue"); + synchronized (eventQueue){ + while (eventQueue.isEmpty() && error == null){ + logger.debug("### No data in the msg queue, waiting... "); + eventQueue.wait(); + } + + logger.debug("### Get notified from consummer thread"); + Tuple2> events = eventQueue.poll(); + if (events != null && events.f0 != null && events.f1 != null){ + logger.debug("### Get event data from {}", events.f0.toString()); + int queueSize = eventQueue.size(); + if (queueSize < MAX_EVENTS_BLOCK_IN_QUEUE / 2){ + eventQueue.notifyAll(); + } + return events; + } + else { + ExceptionUtils.rethrowException(error, error.getMessage()); + return null; + } + } + } + + public void produce(final Tuple2> events) throws InterruptedException{ + if (events == null || events.f0 == null || events.f1 == null){ + logger.error("Received empty events from event producer"); + return; + } + + synchronized (eventQueue){ + while (eventQueue.size() > MAX_EVENTS_BLOCK_IN_QUEUE){ + logger.warn("Event queue is full, current size is {}", eventQueue.size()); + eventQueue.wait(); + } + + eventQueue.add(events); + eventQueue.notifyAll(); + logger.debug("Add received events into queue"); + } + } + + @Override + public void close() { + synchronized (eventQueue){ + logger.info("Close handover on demand"); + eventQueue.clear(); + if (error == null){ + error = new Throwable("Handover closed on command"); + } + + eventQueue.notifyAll(); + } + } + + public void reportError(Throwable t) { + if (t == null){ + return; + } + + synchronized (eventQueue){ + if (error == null){ + error = t; + } + eventQueue.clear(); + eventQueue.notifyAll(); + logger.info("Consumer thread report a error: {}", error.getMessage()); + } + } + + public void wakeupProducer() { + synchronized (eventQueue){ + logger.info("Wakeup producer on demand"); + eventQueue.clear(); + eventQueue.notifyAll(); + } + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java new file mode 100644 index 0000000000000..35d8e300c487f --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java @@ -0,0 +1,124 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.util.ExceptionUtils; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Date; +import java.util.concurrent.ArrayBlockingQueue; + +/** + * Created by jozh on 6/20/2017. + */ +public final class ProducerCache implements Closeable, Serializable { + private static final Logger logger = LoggerFactory.getLogger(ProducerCache.class); + private static final long defaultCheckQueueStatusInterval = 50; + public static final int DEFAULTCAPACITY = 100; + public static final long DEFAULTTIMEOUTMILLISECOND = 100; + private final ArrayBlockingQueue cacheQueue; + private final int queueCapacity; + private final long pollTimeout; + private Date lastPollTime; + private Throwable error; + private volatile boolean closed; + + public ProducerCache(){ + this(DEFAULTCAPACITY, DEFAULTTIMEOUTMILLISECOND); + } + + public ProducerCache(int capacity){ + this(capacity, DEFAULTTIMEOUTMILLISECOND); + } + + public ProducerCache(int capacity, long timeout){ + this.queueCapacity = capacity; + this.pollTimeout = timeout; + this.cacheQueue = new ArrayBlockingQueue(this.queueCapacity); + this.lastPollTime = new Date(); + this.closed = false; + } + + public void put(EventData value) throws InterruptedException{ + if (value == null){ + logger.error("Received empty events from event producer"); + return; + } + + synchronized (cacheQueue){ + while (cacheQueue.remainingCapacity() <= 0 && !closed){ + logger.warn("Event queue is full, current size is {}", cacheQueue.size()); + cacheQueue.wait(); + } + + if (closed){ + logger.info("Cache is closed, event is dropped."); + return; + } + + cacheQueue.add(value); + cacheQueue.notifyAll(); + + logger.debug("Add event into queue"); + } + } + + public ArrayList pollNextBatch() throws InterruptedException{ + logger.debug("###Begin to poll all data from event cache queue"); + + synchronized (cacheQueue){ + while (!isPollTimeout() && !closed && cacheQueue.remainingCapacity() > 0){ + cacheQueue.wait(defaultCheckQueueStatusInterval); + } + + final ArrayList result = new ArrayList<>(cacheQueue.size()); + for (EventData item : cacheQueue){ + result.add(item); + } + cacheQueue.clear(); + cacheQueue.notifyAll(); + + lastPollTime = new Date(); + return result; + } + } + + public void reportError(Throwable t) { + if (t == null){ + return; + } + + synchronized (cacheQueue){ + if (error == null){ + error = t; + } + logger.info("Producer thread report a error: {}", t.toString()); + } + } + + @Override + public void close() { + synchronized (cacheQueue){ + logger.info("Close cache on demand"); + closed = true; + cacheQueue.notifyAll(); + } + } + + public void checkErr() throws Exception { + synchronized (cacheQueue){ + if (error != null){ + ExceptionUtils.rethrowException(error, error.getMessage()); + } + } + } + + private boolean isPollTimeout(){ + long pollInterval = (new Date()).getTime() - lastPollTime.getTime(); + return pollInterval > pollTimeout; + } +} diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index bc3f82f686c44..2ed3b7974098b 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -54,6 +54,7 @@ under the License. flink-connector-nifi flink-connector-cassandra flink-connector-filesystem + flink-connector-eventhubs + +This page briefly discusses how to test Flink application in the local environment. + +* This will be replaced by the TOC +{:toc} + +## Unit testing + +It is encouraged to test your classes with unit tests as much as possible. For example if one implement following `ReduceFunction`: + +~~~java +public class SumReduce implements ReduceFunction { + @Override + public Long reduce(Long value1, Long value2) throws Exception { + return value1 + value2; + } +} +~~~ + +it is very easy to unit test it with your favorite framework: + +~~~java +public class SumReduceTest { + @Test + public void testSum() throws Exception { + SumReduce sumReduce = new SumReduce(); + + assertEquals(42L, sumReduce.reduce(40L, 2L)); + } +} +~~~ + +Or in scala: + +~~~scala +class SumReduce extends ReduceFunction[Long] { + override def reduce(value1: java.lang.Long, + value2: java.lang.Long): java.lang.Long = value1 + value2 +} +~~~ + +~~~scala +class SumReduceTest extends FlatSpec with Matchers { + "SumReduce" should "add values" in { + val sumReduce: SumReduce = new SumReduce() + sumReduce.reduce(40L, 2L) should be (42L) + } +} +~~~ + +## Integration testing + +You also can write integration tests that are executed against local Flink mini cluster. +In order to do so add a test dependency `flink-test-utils`. + +{% highlight xml %} + + org.apache.flink + flink-test-utils{{ site.scala_version_suffix }} + {{site.version }} + +{% endhighlight %} + +For example if you want to test the following `MapFunction`: + +~~~java +public class MultiplyByTwo implements MapFunction { + @Override + public Long map(Long value) throws Exception { + return value * 2; + } +} +~~~ + +You could write following integration test: + +~~~java +public class ExampleIntegrationTest extends StreamingMultipleProgramsTestBase { + @Test + public void testMultiply() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + // values are collected on a static variable + CollectSink.values.clear(); + + env.fromElements(1L, 21L, 22L) + .map(new MultiplyByTwo()) + .addSink(new CollectSink()); + env.execute(); + + assertEquals(Lists.newArrayList(2L, 42L, 44L), CollectSink.values); + } + + private static class CollectSink implements SinkFunction { + // must be static + public static final List values = new ArrayList<>(); + + @Override + public synchronized void invoke(Long value) throws Exception { + values.add(value); + } + } +} +~~~ + +or in Scala: + +~~~scala +class MultiplyByTwo extends MapFunction[Long, Long] { + override def map(value: java.lang.Long): java.lang.Long = value * 2 +} +~~~ + +~~~scala +class ExampleIntegrationTest extends FlatSpec with Matchers { + "MultiplyByTwo" should "multiply it input by two" in { + val env: StreamExecutionEnvironment = + StreamExecutionEnvironment.getExecutionEnvironment + env.setParallelism(1) + // values are collected on a static variable + CollectSink.values.clear() + env + .fromElements(1L, 21L, 22L) + .map(new MultiplyByTwo()) + .addSink(new CollectSink()) + env.execute() + CollectSink.values should be (Lists.newArrayList(2L, 42L, 44L)) + } +} + +object CollectSink { + // must be static + val values: List[Long] = new ArrayList() +} + +class CollectSink extends SinkFunction[Long] { + override def invoke(value: java.lang.Long): Unit = { + synchronized { + values.add(value) + } + } +} +~~~ + +Static variable in `CollectSink` is used here because Flink serializes all operators before distributing them across a cluster. +Communicating with operators instantiated by a local flink mini cluster via static variables is one way around this issue. +Alternatively in your test sink you could for example write the data to files in a temporary directory. +Of course you could use your own custom sources and sinks, which can emit watermarks. + +## Testing checkpointing and state handling + +One way to test state handling is to enable checkpointing in integration tests. You can do that by +configuring `environment` in the test: +~~~java +env.enableCheckpointing(500); +env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 100)); +~~~ +and for example adding to your Flink application an identity mapper operator that will throw an exception +once every `1000ms`. However writing such test could be tricky because of time dependencies between the actions. + +Another approach is to write a unit test using `AbstractStreamOperatorTestHarness` from `flink-streaming-java` module. +For example how to do that please look at the `org.apache.flink.streaming.runtime.operators.windowing.WindowOperatorTest` +also in the `flink-streaming-java`. Be aware that `AbstractStreamOperatorTestHarness` is not currently a part of public API +and can be subject to change. From 5ce0ded1b71faf7e5d90865ba9481e254c3fb5be Mon Sep 17 00:00:00 2001 From: twalthr Date: Tue, 29 Aug 2017 17:19:50 +0200 Subject: [PATCH 062/129] [hotfix] [docs] Fix typos and improve testing docs --- docs/dev/stream/testing.md | 263 +++++++++++++++++++++++++++++++++++++ 1 file changed, 263 insertions(+) create mode 100644 docs/dev/stream/testing.md diff --git a/docs/dev/stream/testing.md b/docs/dev/stream/testing.md new file mode 100644 index 0000000000000..44f5cfd5d9a12 --- /dev/null +++ b/docs/dev/stream/testing.md @@ -0,0 +1,263 @@ +--- +title: "Testing" +nav-parent_id: streaming +nav-id: testing +nav-pos: 99 +--- + + +This page briefly discusses how to test a Flink application in your IDE or a local environment. + +* This will be replaced by the TOC +{:toc} + +## Unit testing + +Usually, one can assume that Flink produces correct results outside of a user-defined `Function`. Therefore, it is recommended to test `Function` classes that contain the main business logic with unit tests as much as possible. + +For example if one implements the following `ReduceFunction`: + +

+
+{% highlight java %} +public class SumReduce implements ReduceFunction { + + @Override + public Long reduce(Long value1, Long value2) throws Exception { + return value1 + value2; + } +} +{% endhighlight %} +
+ +
+{% highlight scala %} +class SumReduce extends ReduceFunction[Long] { + + override def reduce(value1: java.lang.Long, value2: java.lang.Long): java.lang.Long = { + value1 + value2 + } +} +{% endhighlight %} +
+
+ +It is very easy to unit test it with your favorite framework by passing suitable arguments and verify the output: + +
+
+{% highlight java %} +public class SumReduceTest { + + @Test + public void testSum() throws Exception { + // intiantiate your function + SumReduce sumReduce = new SumReduce(); + + // call the methods that you have implemented + assertEquals(42L, sumReduce.reduce(40L, 2L)); + } +} +{% endhighlight %} +
+ +
+{% highlight scala %} +class SumReduceTest extends FlatSpec with Matchers { + + "SumReduce" should "add values" in { + // intiantiate your function + val sumReduce: SumReduce = new SumReduce() + + // call the methods that you have implemented + sumReduce.reduce(40L, 2L) should be (42L) + } +} +{% endhighlight %} +
+
+ +## Integration testing + +In order to end-to-end test Flink streaming pipelines, you can also write integration tests that are executed against a local Flink mini cluster. + +In order to do so add the test dependency `flink-test-utils`: + +{% highlight xml %} + + org.apache.flink + flink-test-utils{{ site.scala_version_suffix }} + {{site.version }} + +{% endhighlight %} + +For example, if you want to test the following `MapFunction`: + +
+
+{% highlight java %} +public class MultiplyByTwo implements MapFunction { + + @Override + public Long map(Long value) throws Exception { + return value * 2; + } +} +{% endhighlight %} +
+ +
+{% highlight scala %} +class MultiplyByTwo extends MapFunction[Long, Long] { + + override def map(value: Long): Long = { + value * 2 + } +} +{% endhighlight %} +
+
+ +You could write the following integration test: + +
+
+{% highlight java %} +public class ExampleIntegrationTest extends StreamingMultipleProgramsTestBase { + + @Test + public void testMultiply() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + // configure your test environment + env.setParallelism(1); + + // values are collected in a static variable + CollectSink.values.clear(); + + // create a stream of custom elements and apply transformations + env.fromElements(1L, 21L, 22L) + .map(new MultiplyByTwo()) + .addSink(new CollectSink()); + + // execute + env.execute(); + + // verify your results + assertEquals(Lists.newArrayList(2L, 42L, 44L), CollectSink.values); + } + + // create a testing sink + private static class CollectSink implements SinkFunction { + + // must be static + public static final List values = new ArrayList<>(); + + @Override + public synchronized void invoke(Long value) throws Exception { + values.add(value); + } + } +} +{% endhighlight %} +
+ +
+{% highlight scala %} +class ExampleIntegrationTest extends StreamingMultipleProgramsTestBase { + + @Test + def testMultiply(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + + // configure your test environment + env.setParallelism(1) + + // values are collected in a static variable + CollectSink.values.clear() + + // create a stream of custom elements and apply transformations + env + .fromElements(1L, 21L, 22L) + .map(new MultiplyByTwo()) + .addSink(new CollectSink()) + + // execute + env.execute() + + // verify your results + assertEquals(Lists.newArrayList(2L, 42L, 44L), CollectSink.values) + } +} + +// create a testing sink +class CollectSink extends SinkFunction[Long] { + + override def invoke(value: java.lang.Long): Unit = { + synchronized { + values.add(value) + } + } +} + +object CollectSink { + + // must be static + val values: List[Long] = new ArrayList() +} +{% endhighlight %} +
+
+ +The static variable in `CollectSink` is used here because Flink serializes all operators before distributing them across a cluster. +Communicating with operators instantiated by a local Flink mini cluster via static variables is one way around this issue. +Alternatively, you could for example write the data to files in a temporary directory with your test sink. +You can also implement your own custom sources for emitting watermarks. + +## Testing checkpointing and state handling + +One way to test state handling is to enable checkpointing in integration tests. + +You can do that by configuring your `StreamExecutionEnvironment` in the test: + +
+
+{% highlight java %} +env.enableCheckpointing(500); +env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 100)); +{% endhighlight %} +
+ +
+{% highlight scala %} +env.enableCheckpointing(500); +env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 100)); +{% endhighlight %} +
+
+ +And for example adding to your Flink application an identity mapper operator that will throw an exception +once every `1000ms`. However writing such test could be tricky because of time dependencies between the actions. + +Another approach is to write a unit test using the Flink internal testing utility `AbstractStreamOperatorTestHarness` from the `flink-streaming-java` module. + +For an example of how to do that please have a look at the `org.apache.flink.streaming.runtime.operators.windowing.WindowOperatorTest` also in the `flink-streaming-java` module. + +Be aware that `AbstractStreamOperatorTestHarness` is currently not a part of public API and can be subject to change. From 88848e74cfade38737f306aeba29a4fe9e26ba79 Mon Sep 17 00:00:00 2001 From: twalthr Date: Tue, 29 Aug 2017 17:27:33 +0200 Subject: [PATCH 063/129] [hotfix] [docs] Remove duplicate docs page --- docs/dev/testing.md | 189 -------------------------------------------- 1 file changed, 189 deletions(-) delete mode 100644 docs/dev/testing.md diff --git a/docs/dev/testing.md b/docs/dev/testing.md deleted file mode 100644 index df65306213446..0000000000000 --- a/docs/dev/testing.md +++ /dev/null @@ -1,189 +0,0 @@ ---- -title: "Testing" -nav-parent_id: dev -nav-id: testing -nav-pos: 99 ---- - - -This page briefly discusses how to test Flink application in the local environment. - -* This will be replaced by the TOC -{:toc} - -## Unit testing - -It is encouraged to test your classes with unit tests as much as possible. For example if one implement following `ReduceFunction`: - -~~~java -public class SumReduce implements ReduceFunction { - @Override - public Long reduce(Long value1, Long value2) throws Exception { - return value1 + value2; - } -} -~~~ - -it is very easy to unit test it with your favorite framework: - -~~~java -public class SumReduceTest { - @Test - public void testSum() throws Exception { - SumReduce sumReduce = new SumReduce(); - - assertEquals(42L, sumReduce.reduce(40L, 2L)); - } -} -~~~ - -Or in scala: - -~~~scala -class SumReduce extends ReduceFunction[Long] { - override def reduce(value1: java.lang.Long, - value2: java.lang.Long): java.lang.Long = value1 + value2 -} -~~~ - -~~~scala -class SumReduceTest extends FlatSpec with Matchers { - "SumReduce" should "add values" in { - val sumReduce: SumReduce = new SumReduce() - sumReduce.reduce(40L, 2L) should be (42L) - } -} -~~~ - -## Integration testing - -You also can write integration tests that are executed against local Flink mini cluster. -In order to do so add a test dependency `flink-test-utils`. - -{% highlight xml %} - - org.apache.flink - flink-test-utils{{ site.scala_version_suffix }} - {{site.version }} - -{% endhighlight %} - -For example if you want to test the following `MapFunction`: - -~~~java -public class MultiplyByTwo implements MapFunction { - @Override - public Long map(Long value) throws Exception { - return value * 2; - } -} -~~~ - -You could write following integration test: - -~~~java -public class ExampleIntegrationTest extends StreamingMultipleProgramsTestBase { - @Test - public void testMultiply() throws Exception { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - // values are collected on a static variable - CollectSink.values.clear(); - - env.fromElements(1L, 21L, 22L) - .map(new MultiplyByTwo()) - .addSink(new CollectSink()); - env.execute(); - - assertEquals(Lists.newArrayList(2L, 42L, 44L), CollectSink.values); - } - - private static class CollectSink implements SinkFunction { - // must be static - public static final List values = new ArrayList<>(); - - @Override - public synchronized void invoke(Long value) throws Exception { - values.add(value); - } - } -} -~~~ - -or in Scala: - -~~~scala -class MultiplyByTwo extends MapFunction[Long, Long] { - override def map(value: java.lang.Long): java.lang.Long = value * 2 -} -~~~ - -~~~scala -class ExampleIntegrationTest extends FlatSpec with Matchers { - "MultiplyByTwo" should "multiply it input by two" in { - val env: StreamExecutionEnvironment = - StreamExecutionEnvironment.getExecutionEnvironment - env.setParallelism(1) - // values are collected on a static variable - CollectSink.values.clear() - env - .fromElements(1L, 21L, 22L) - .map(new MultiplyByTwo()) - .addSink(new CollectSink()) - env.execute() - CollectSink.values should be (Lists.newArrayList(2L, 42L, 44L)) - } -} - -object CollectSink { - // must be static - val values: List[Long] = new ArrayList() -} - -class CollectSink extends SinkFunction[Long] { - override def invoke(value: java.lang.Long): Unit = { - synchronized { - values.add(value) - } - } -} -~~~ - -Static variable in `CollectSink` is used here because Flink serializes all operators before distributing them across a cluster. -Communicating with operators instantiated by a local flink mini cluster via static variables is one way around this issue. -Alternatively in your test sink you could for example write the data to files in a temporary directory. -Of course you could use your own custom sources and sinks, which can emit watermarks. - -## Testing checkpointing and state handling - -One way to test state handling is to enable checkpointing in integration tests. You can do that by -configuring `environment` in the test: -~~~java -env.enableCheckpointing(500); -env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 100)); -~~~ -and for example adding to your Flink application an identity mapper operator that will throw an exception -once every `1000ms`. However writing such test could be tricky because of time dependencies between the actions. - -Another approach is to write a unit test using `AbstractStreamOperatorTestHarness` from `flink-streaming-java` module. -For example how to do that please look at the `org.apache.flink.streaming.runtime.operators.windowing.WindowOperatorTest` -also in the `flink-streaming-java`. Be aware that `AbstractStreamOperatorTestHarness` is not currently a part of public API -and can be subject to change. From 1fc0b6413c74eff0ace25f4329451e35e84849b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=AE=9D=E7=89=9B?= Date: Wed, 23 Aug 2017 17:45:05 +0800 Subject: [PATCH 064/129] [FLINK-7206] [table] Add DataView to support direct state access in AggregateFunction accumulators. This closes #4355. --- .../flink/table/api/dataview/DataView.scala | 35 +++ .../table/api/dataview/DataViewSpec.scala | 55 +++++ .../flink/table/api/dataview/ListView.scala | 142 +++++++++++ .../flink/table/api/dataview/MapView.scala | 198 +++++++++++++++ .../codegen/AggregationCodeGenerator.scala | 231 +++++++++++++++--- .../flink/table/codegen/CodeGenerator.scala | 27 +- .../table/dataview/ListViewSerializer.scala | 111 +++++++++ .../table/dataview/ListViewTypeInfo.scala | 66 +++++ .../dataview/ListViewTypeInfoFactory.scala | 43 ++++ .../table/dataview/MapViewSerializer.scala | 121 +++++++++ .../table/dataview/MapViewTypeInfo.scala | 72 ++++++ .../dataview/MapViewTypeInfoFactory.scala | 51 ++++ .../flink/table/dataview/StateListView.scala | 47 ++++ .../flink/table/dataview/StateMapView.scala | 54 ++++ .../utils/UserDefinedFunctionUtils.scala | 113 ++++++++- .../runtime/aggregate/AggregateUtil.scala | 95 ++++--- .../aggregate/GeneratedAggregations.scala | 21 +- .../aggregate/GroupAggProcessFunction.scala | 8 +- .../aggregate/ProcTimeBoundedRangeOver.scala | 11 +- .../aggregate/ProcTimeBoundedRowsOver.scala | 11 +- .../aggregate/ProcTimeUnboundedOver.scala | 9 +- .../aggregate/RowTimeBoundedRangeOver.scala | 6 + .../aggregate/RowTimeBoundedRowsOver.scala | 6 + .../aggregate/RowTimeUnboundedOver.scala | 5 + .../utils/JavaUserDefinedAggFunctions.java | 199 +++++++++++++++ .../dataview/ListViewSerializerTest.scala | 62 +++++ .../dataview/MapViewSerializerTest.scala | 68 ++++++ .../runtime/batch/table/AggregateITCase.scala | 9 +- .../runtime/harness/HarnessTestBase.scala | 17 ++ .../stream/table/AggregateITCase.scala | 41 +++- .../stream/table/GroupWindowITCase.scala | 40 +-- .../stream/table/OverWindowITCase.scala | 118 +++++---- 32 files changed, 1933 insertions(+), 159 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/DataView.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/DataViewSpec.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/ListView.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/MapView.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewTypeInfo.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewTypeInfoFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewTypeInfo.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewTypeInfoFactory.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/StateListView.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/StateMapView.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/dataview/ListViewSerializerTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/dataview/MapViewSerializerTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/DataView.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/DataView.scala new file mode 100644 index 0000000000000..2214086e23fa5 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/DataView.scala @@ -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. + */ + +package org.apache.flink.table.api.dataview + +/** + * A [[DataView]] is a collection type that can be used in the accumulator of an + * [[org.apache.flink.table.functions.AggregateFunction]]. + * + * Depending on the context in which the [[org.apache.flink.table.functions.AggregateFunction]] is + * used, a [[DataView]] can be backed by a Java heap collection or a state backend. + */ +trait DataView extends Serializable { + + /** + * Clears the [[DataView]] and removes all data. + */ + def clear(): Unit + +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/DataViewSpec.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/DataViewSpec.scala new file mode 100644 index 0000000000000..943fe033d7016 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/DataViewSpec.scala @@ -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.api.dataview + +import java.lang.reflect.Field + +import org.apache.flink.api.common.state.{ListStateDescriptor, MapStateDescriptor, State, StateDescriptor} +import org.apache.flink.table.dataview.{ListViewTypeInfo, MapViewTypeInfo} + +/** + * Data view specification. + * + * @tparam ACC type extends [[DataView]] + */ +trait DataViewSpec[ACC <: DataView] { + def stateId: String + def field: Field + def toStateDescriptor: StateDescriptor[_ <: State, _] +} + +case class ListViewSpec[T]( + stateId: String, + field: Field, + listViewTypeInfo: ListViewTypeInfo[T]) + extends DataViewSpec[ListView[T]] { + + override def toStateDescriptor: StateDescriptor[_ <: State, _] = + new ListStateDescriptor[T](stateId, listViewTypeInfo.elementType) +} + +case class MapViewSpec[K, V]( + stateId: String, + field: Field, + mapViewTypeInfo: MapViewTypeInfo[K, V]) + extends DataViewSpec[MapView[K, V]] { + + override def toStateDescriptor: StateDescriptor[_ <: State, _] = + new MapStateDescriptor[K, V](stateId, mapViewTypeInfo.keyType, mapViewTypeInfo.valueType) +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/ListView.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/ListView.scala new file mode 100644 index 0000000000000..59b2426db6b11 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/ListView.scala @@ -0,0 +1,142 @@ +/* + * 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.api.dataview + +import java.lang.{Iterable => JIterable} +import java.util + +import org.apache.flink.api.common.typeinfo.{TypeInfo, TypeInformation} +import org.apache.flink.table.dataview.ListViewTypeInfoFactory + +/** + * A [[ListView]] provides List functionality for accumulators used by user-defined aggregate + * functions [[org.apache.flink.api.common.functions.AggregateFunction]]. + * + * A [[ListView]] can be backed by a Java ArrayList or a state backend, depending on the context in + * which the aggregate function is used. + * + * At runtime [[ListView]] will be replaced by a [[org.apache.flink.table.dataview.StateListView]] + * if it is backed by a state backend. + * + * Example of an accumulator type with a [[ListView]] and an aggregate function that uses it: + * {{{ + * + * public class MyAccum { + * public ListView list; + * public long count; + * } + * + * public class MyAgg extends AggregateFunction { + * + * @Override + * public MyAccum createAccumulator() { + * MyAccum accum = new MyAccum(); + * accum.list = new ListView<>(Types.STRING); + * accum.count = 0L; + * return accum; + * } + * + * public void accumulate(MyAccum accumulator, String id) { + * accumulator.list.add(id); + * ... ... + * accumulator.get() + * ... ... + * } + * + * @Override + * public Long getValue(MyAccum accumulator) { + * accumulator.list.add(id); + * ... ... + * accumulator.get() + * ... ... + * return accumulator.count; + * } + * } + * + * }}} + * + * @param elementTypeInfo element type information + * @tparam T element type + */ +@TypeInfo(classOf[ListViewTypeInfoFactory[_]]) +class ListView[T]( + @transient private[flink] val elementTypeInfo: TypeInformation[T], + private[flink] val list: util.List[T]) + extends DataView { + + /** + * Creates a list view for elements of the specified type. + * + * @param elementTypeInfo The type of the list view elements. + */ + def this(elementTypeInfo: TypeInformation[T]) { + this(elementTypeInfo, new util.ArrayList[T]()) + } + + /** + * Creates a list view. + */ + def this() = this(null) + + /** + * Returns an iterable of the list view. + * + * @throws Exception Thrown if the system cannot get data. + * @return The iterable of the list or { @code null} if the list is empty. + */ + @throws[Exception] + def get: JIterable[T] = { + if (!list.isEmpty) { + list + } else { + null + } + } + + /** + * Adds the given value to the list. + * + * @throws Exception Thrown if the system cannot add data. + * @param value The element to be appended to this list view. + */ + @throws[Exception] + def add(value: T): Unit = list.add(value) + + /** + * Adds all of the elements of the specified list to this list view. + * + * @throws Exception Thrown if the system cannot add all data. + * @param list The list with the elements that will be stored in this list view. + */ + @throws[Exception] + def addAll(list: util.List[T]): Unit = this.list.addAll(list) + + /** + * Removes all of the elements from this list view. + */ + override def clear(): Unit = list.clear() + + override def equals(other: Any): Boolean = other match { + case that: ListView[T] => + list.equals(that.list) + case _ => false + } + + override def hashCode(): Int = list.hashCode() +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/MapView.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/MapView.scala new file mode 100644 index 0000000000000..9206d6af3e4a1 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/dataview/MapView.scala @@ -0,0 +1,198 @@ +/* + * 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.api.dataview + +import java.lang.{Iterable => JIterable} +import java.util + +import org.apache.flink.api.common.typeinfo.{TypeInfo, TypeInformation} +import org.apache.flink.table.dataview.MapViewTypeInfoFactory + +/** + * A [[MapView]] provides Map functionality for accumulators used by user-defined aggregate + * functions [[org.apache.flink.table.functions.AggregateFunction]]. + * + * A [[MapView]] can be backed by a Java HashMap or a state backend, depending on the context in + * which the aggregation function is used. + * + * At runtime [[MapView]] will be replaced by a [[org.apache.flink.table.dataview.StateMapView]] + * if it is backed by a state backend. + * + * Example of an accumulator type with a [[MapView]] and an aggregate function that uses it: + * {{{ + * + * public class MyAccum { + * public MapView map; + * public long count; + * } + * + * public class MyAgg extends AggregateFunction { + * + * @Override + * public MyAccum createAccumulator() { + * MyAccum accum = new MyAccum(); + * accum.map = new MapView<>(Types.STRING, Types.INT); + * accum.count = 0L; + * return accum; + * } + * + * public void accumulate(MyAccum accumulator, String id) { + * try { + * if (!accumulator.map.contains(id)) { + * accumulator.map.put(id, 1); + * accumulator.count++; + * } + * } catch (Exception e) { + * e.printStackTrace(); + * } + * } + * + * @Override + * public Long getValue(MyAccum accumulator) { + * return accumulator.count; + * } + * } + * + * }}} + * + * @param keyTypeInfo key type information + * @param valueTypeInfo value type information + * @tparam K key type + * @tparam V value type + */ +@TypeInfo(classOf[MapViewTypeInfoFactory[_, _]]) +class MapView[K, V]( + @transient private[flink] val keyTypeInfo: TypeInformation[K], + @transient private[flink] val valueTypeInfo: TypeInformation[V], + private[flink] val map: util.Map[K, V]) + extends DataView { + + /** + * Creates a MapView with the specified key and value types. + * + * @param keyTypeInfo The type of keys of the MapView. + * @param valueTypeInfo The type of the values of the MapView. + */ + def this(keyTypeInfo: TypeInformation[K], valueTypeInfo: TypeInformation[V]) { + this(keyTypeInfo, valueTypeInfo, new util.HashMap[K, V]()) + } + + /** + * Creates a MapView. + */ + def this() = this(null, null) + + /** + * Return the value for the specified key or { @code null } if the key is not in the map view. + * + * @param key The look up key. + * @return The value for the specified key. + * @throws Exception Thrown if the system cannot get data. + */ + @throws[Exception] + def get(key: K): V = map.get(key) + + /** + * Inserts a value for the given key into the map view. + * If the map view already contains a value for the key, the existing value is overwritten. + * + * @param key The key for which the value is inserted. + * @param value The value that is inserted for the key. + * @throws Exception Thrown if the system cannot put data. + */ + @throws[Exception] + def put(key: K, value: V): Unit = map.put(key, value) + + /** + * Inserts all mappings from the specified map to this map view. + * + * @param map The map whose entries are inserted into this map view. + * @throws Exception Thrown if the system cannot access the map. + */ + @throws[Exception] + def putAll(map: util.Map[K, V]): Unit = this.map.putAll(map) + + /** + * Deletes the value for the given key. + * + * @param key The key for which the value is deleted. + * @throws Exception Thrown if the system cannot access the map. + */ + @throws[Exception] + def remove(key: K): Unit = map.remove(key) + + /** + * Checks if the map view contains a value for a given key. + * + * @param key The key to check. + * @return True if there exists a value for the given key, false otherwise. + * @throws Exception Thrown if the system cannot access the map. + */ + @throws[Exception] + def contains(key: K): Boolean = map.containsKey(key) + + /** + * Returns all entries of the map view. + * + * @return An iterable of all the key-value pairs in the map view. + * @throws Exception Thrown if the system cannot access the map. + */ + @throws[Exception] + def entries: JIterable[util.Map.Entry[K, V]] = map.entrySet() + + /** + * Returns all the keys in the map view. + * + * @return An iterable of all the keys in the map. + * @throws Exception Thrown if the system cannot access the map. + */ + @throws[Exception] + def keys: JIterable[K] = map.keySet() + + /** + * Returns all the values in the map view. + * + * @return An iterable of all the values in the map. + * @throws Exception Thrown if the system cannot access the map. + */ + @throws[Exception] + def values: JIterable[V] = map.values() + + /** + * Returns an iterator over all entries of the map view. + * + * @return An iterator over all the mappings in the map. + * @throws Exception Thrown if the system cannot access the map. + */ + @throws[Exception] + def iterator: util.Iterator[util.Map.Entry[K, V]] = map.entrySet().iterator() + + /** + * Removes all entries of this map. + */ + override def clear(): Unit = map.clear() + + override def equals(other: Any): Boolean = other match { + case that: MapView[K, V] => + map.equals(that.map) + case _ => false + } + + override def hashCode(): Int = map.hashCode() +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala index 25527ccf3c2cf..22ce5ba4a317f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/AggregationCodeGenerator.scala @@ -17,17 +17,23 @@ */ package org.apache.flink.table.codegen -import java.lang.reflect.ParameterizedType +import java.lang.reflect.{Modifier, ParameterizedType} import java.lang.{Iterable => JIterable} +import org.apache.commons.codec.binary.Base64 +import org.apache.flink.api.common.state.{State, StateDescriptor} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.api.dataview._ import org.apache.flink.table.codegen.Indenter.toISC -import org.apache.flink.table.codegen.CodeGenUtils.newName +import org.apache.flink.table.codegen.CodeGenUtils.{newName, reflectiveFieldWriteAccess} import org.apache.flink.table.functions.AggregateFunction import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{getUserDefinedMethod, signatureToString} import org.apache.flink.table.runtime.aggregate.{GeneratedAggregations, SingleElementIterable} +import org.apache.flink.util.InstantiationUtil + +import scala.collection.mutable /** * A code generator for generating [[GeneratedAggregations]]. @@ -42,6 +48,18 @@ class AggregationCodeGenerator( input: TypeInformation[_ <: Any]) extends CodeGenerator(config, nullableInput, input) { + // set of statements for cleanup dataview that will be added only once + // we use a LinkedHashSet to keep the insertion order + private val reusableCleanupStatements = mutable.LinkedHashSet[String]() + + /** + * @return code block of statements that need to be placed in the cleanup() method of + * [[GeneratedAggregations]] + */ + def reuseCleanupCode(): String = { + reusableCleanupStatements.mkString("", "\n", "\n") + } + /** * Generates a [[org.apache.flink.table.runtime.aggregate.GeneratedAggregations]] that can be * passed to a Java compiler. @@ -79,13 +97,15 @@ class AggregationCodeGenerator( outputArity: Int, needRetract: Boolean, needMerge: Boolean, - needReset: Boolean) + needReset: Boolean, + accConfig: Option[Array[Seq[DataViewSpec[_]]]]) : GeneratedAggregationsFunction = { // get unique function name val funcName = newName(name) // register UDAGGs - val aggs = aggregates.map(a => addReusableFunction(a)) + val aggs = aggregates.map(a => addReusableFunction(a, contextTerm)) + // get java types of accumulators val accTypeClasses = aggregates.map { a => a.getClass.getMethod("createAccumulator").getReturnType @@ -105,6 +125,9 @@ class AggregationCodeGenerator( inFields.map(classes(_)) } + // initialize and create data views + addReusableDataViews() + // check and validate the needed methods aggregates.zipWithIndex.map { case (a, i) => @@ -161,13 +184,121 @@ class AggregationCodeGenerator( } } + /** + * Create DataView Term, for example, acc1_map_dataview. + * + * @param aggIndex index of aggregate function + * @param fieldName field name of DataView + * @return term to access [[MapView]] or [[ListView]] + */ + def createDataViewTerm(aggIndex: Int, fieldName: String): String = { + s"acc${aggIndex}_${fieldName}_dataview" + } + + /** + * Adds a reusable [[org.apache.flink.table.api.dataview.DataView]] to the open, cleanup, + * close and member area of the generated function. + * + */ + def addReusableDataViews(): Unit = { + if (accConfig.isDefined) { + val descMapping: Map[String, StateDescriptor[_, _]] = accConfig.get + .flatMap(specs => specs.map(s => (s.stateId, s.toStateDescriptor))) + .toMap[String, StateDescriptor[_ <: State, _]] + + for (i <- aggs.indices) yield { + for (spec <- accConfig.get(i)) yield { + val dataViewField = spec.field + val dataViewTypeTerm = dataViewField.getType.getCanonicalName + val desc = descMapping.getOrElse(spec.stateId, + throw new CodeGenException( + s"Can not find DataView in accumulator by id: ${spec.stateId}")) + + // define the DataView variables + val serializedData = serializeStateDescriptor(desc) + val dataViewFieldTerm = createDataViewTerm(i, dataViewField.getName) + val field = + s""" + | transient $dataViewTypeTerm $dataViewFieldTerm = null; + |""".stripMargin + reusableMemberStatements.add(field) + + // create DataViews + val descFieldTerm = s"${dataViewFieldTerm}_desc" + val descClassQualifier = classOf[StateDescriptor[_, _]].getCanonicalName + val descDeserializeCode = + s""" + | $descClassQualifier $descFieldTerm = ($descClassQualifier) + | org.apache.flink.util.InstantiationUtil.deserializeObject( + | org.apache.commons.codec.binary.Base64.decodeBase64("$serializedData"), + | $contextTerm.getUserCodeClassLoader()); + |""".stripMargin + val createDataView = if (dataViewField.getType == classOf[MapView[_, _]]) { + s""" + | $descDeserializeCode + | $dataViewFieldTerm = new org.apache.flink.table.dataview.StateMapView( + | $contextTerm.getMapState(( + | org.apache.flink.api.common.state.MapStateDescriptor)$descFieldTerm)); + |""".stripMargin + } else if (dataViewField.getType == classOf[ListView[_]]) { + s""" + | $descDeserializeCode + | $dataViewFieldTerm = new org.apache.flink.table.dataview.StateListView( + | $contextTerm.getListState(( + | org.apache.flink.api.common.state.ListStateDescriptor)$descFieldTerm)); + |""".stripMargin + } else { + throw new CodeGenException(s"Unsupported dataview type: $dataViewTypeTerm") + } + reusableOpenStatements.add(createDataView) + + // cleanup DataViews + val cleanup = + s""" + | $dataViewFieldTerm.clear(); + |""".stripMargin + reusableCleanupStatements.add(cleanup) + } + } + } + } + + /** + * Generate statements to set data view field when use state backend. + * + * @param accTerm aggregation term + * @param aggIndex index of aggregation + * @return data view field set statements + */ + def genDataViewFieldSetter(accTerm: String, aggIndex: Int): String = { + if (accConfig.isDefined) { + val setters = for (spec <- accConfig.get(aggIndex)) yield { + val field = spec.field + val dataViewTerm = createDataViewTerm(aggIndex, field.getName) + val fieldSetter = if (Modifier.isPublic(field.getModifiers)) { + s"$accTerm.${field.getName} = $dataViewTerm;" + } else { + val fieldTerm = addReusablePrivateFieldAccess(field.getDeclaringClass, field.getName) + s"${reflectiveFieldWriteAccess(fieldTerm, field, accTerm, dataViewTerm)};" + } + + s""" + | $fieldSetter + """.stripMargin + } + setters.mkString("\n") + } else { + "" + } + } + def genSetAggregationResults: String = { val sig: String = j""" | public final void setAggregationResults( | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row output)""".stripMargin + | org.apache.flink.types.Row output) throws Exception """.stripMargin val setAggs: String = { for (i <- aggs.indices) yield @@ -181,10 +312,11 @@ class AggregationCodeGenerator( j""" | org.apache.flink.table.functions.AggregateFunction baseClass$i = | (org.apache.flink.table.functions.AggregateFunction) ${aggs(i)}; - | + | ${accTypes(i)} acc$i = (${accTypes(i)}) accs.getField($i); + | ${genDataViewFieldSetter(s"acc$i", i)} | output.setField( | ${aggMapping(i)}, - | baseClass$i.getValue((${accTypes(i)}) accs.getField($i)));""".stripMargin + | baseClass$i.getValue(acc$i));""".stripMargin } }.mkString("\n") @@ -200,14 +332,17 @@ class AggregationCodeGenerator( j""" | public final void accumulate( | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row input)""".stripMargin + | org.apache.flink.types.Row input) throws Exception """.stripMargin val accumulate: String = { - for (i <- aggs.indices) yield + for (i <- aggs.indices) yield { j""" + | ${accTypes(i)} acc$i = (${accTypes(i)}) accs.getField($i); + | ${genDataViewFieldSetter(s"acc$i", i)} | ${aggs(i)}.accumulate( - | ((${accTypes(i)}) accs.getField($i)), + | acc$i, | ${parametersCode(i)});""".stripMargin + } }.mkString("\n") j"""$sig { @@ -221,14 +356,17 @@ class AggregationCodeGenerator( j""" | public final void retract( | org.apache.flink.types.Row accs, - | org.apache.flink.types.Row input)""".stripMargin + | org.apache.flink.types.Row input) throws Exception """.stripMargin val retract: String = { - for (i <- aggs.indices) yield + for (i <- aggs.indices) yield { j""" + | ${accTypes(i)} acc$i = (${accTypes(i)}) accs.getField($i); + | ${genDataViewFieldSetter(s"acc$i", i)} | ${aggs(i)}.retract( - | ((${accTypes(i)}) accs.getField($i)), + | acc$i, | ${parametersCode(i)});""".stripMargin + } }.mkString("\n") if (needRetract) { @@ -247,7 +385,7 @@ class AggregationCodeGenerator( val sig: String = j""" - | public final org.apache.flink.types.Row createAccumulators() + | public final org.apache.flink.types.Row createAccumulators() throws Exception | """.stripMargin val init: String = j""" @@ -255,12 +393,15 @@ class AggregationCodeGenerator( | new org.apache.flink.types.Row(${aggs.length});""" .stripMargin val create: String = { - for (i <- aggs.indices) yield + for (i <- aggs.indices) yield { j""" + | ${accTypes(i)} acc$i = (${accTypes(i)}) ${aggs(i)}.createAccumulator(); + | ${genDataViewFieldSetter(s"acc$i", i)} | accs.setField( | $i, - | ${aggs(i)}.createAccumulator());""" - .stripMargin + | acc$i);""" + .stripMargin + } }.mkString("\n") val ret: String = j""" @@ -356,6 +497,10 @@ class AggregationCodeGenerator( """.stripMargin if (needMerge) { + if (accConfig.isDefined) { + throw new CodeGenException("DataView doesn't support merge when the backend uses " + + s"state when generate aggregation for $funcName.") + } j""" |$sig { |$merge @@ -385,13 +530,15 @@ class AggregationCodeGenerator( val sig: String = j""" | public final void resetAccumulator( - | org.apache.flink.types.Row accs)""".stripMargin + | org.apache.flink.types.Row accs) throws Exception """.stripMargin val reset: String = { - for (i <- aggs.indices) yield + for (i <- aggs.indices) yield { j""" - | ${aggs(i)}.resetAccumulator( - | ((${accTypes(i)}) accs.getField($i)));""".stripMargin + | ${accTypes(i)} acc$i = (${accTypes(i)}) accs.getField($i); + | ${genDataViewFieldSetter(s"acc$i", i)} + | ${aggs(i)}.resetAccumulator(acc$i);""".stripMargin + } }.mkString("\n") if (needReset) { @@ -404,6 +551,17 @@ class AggregationCodeGenerator( } } + val aggFuncCode = Seq( + genSetAggregationResults, + genAccumulate, + genRetract, + genCreateAccumulators, + genSetForwardedFields, + genSetConstantFlags, + genCreateOutputRow, + genMergeAccumulatorsPair, + genResetAccumulator).mkString("\n") + val generatedAggregationsClass = classOf[GeneratedAggregations].getCanonicalName var funcCode = j""" @@ -416,20 +574,29 @@ class AggregationCodeGenerator( | } | ${reuseConstructorCode(funcName)} | + | public final void open( + | org.apache.flink.api.common.functions.RuntimeContext $contextTerm) throws Exception { + | ${reuseOpenCode()} + | } + | + | $aggFuncCode + | + | public final void cleanup() throws Exception { + | ${reuseCleanupCode()} + | } + | + | public final void close() throws Exception { + | ${reuseCloseCode()} + | } + |} """.stripMargin - funcCode += genSetAggregationResults + "\n" - funcCode += genAccumulate + "\n" - funcCode += genRetract + "\n" - funcCode += genCreateAccumulators + "\n" - funcCode += genSetForwardedFields + "\n" - funcCode += genSetConstantFlags + "\n" - funcCode += genCreateOutputRow + "\n" - funcCode += genMergeAccumulatorsPair + "\n" - funcCode += genResetAccumulator + "\n" - funcCode += "}" - GeneratedAggregationsFunction(funcName, funcCode) } + @throws[Exception] + def serializeStateDescriptor(stateDescriptor: StateDescriptor[_, _]): String = { + val byteArray = InstantiationUtil.serializeObject(stateDescriptor) + Base64.encodeBase64URLSafeString(byteArray) + } } 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 946c6cdad7c96..154e8ad2341ad 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 @@ -41,8 +41,9 @@ import org.apache.flink.table.codegen.calls.FunctionGenerator import org.apache.flink.table.codegen.calls.ScalarOperators._ import org.apache.flink.table.functions.sql.{ProctimeSqlFunction, ScalarSqlFunctions} import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils -import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} + import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo +import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} import org.apache.flink.table.typeutils.TypeCheckUtils._ import scala.collection.JavaConversions._ @@ -108,31 +109,31 @@ abstract class CodeGenerator( // set of member statements that will be added only once // we use a LinkedHashSet to keep the insertion order - private val reusableMemberStatements = mutable.LinkedHashSet[String]() + protected val reusableMemberStatements = mutable.LinkedHashSet[String]() // set of constructor statements that will be added only once // we use a LinkedHashSet to keep the insertion order - private val reusableInitStatements = mutable.LinkedHashSet[String]() + protected val reusableInitStatements = mutable.LinkedHashSet[String]() // set of open statements for RichFunction that will be added only once // we use a LinkedHashSet to keep the insertion order - private val reusableOpenStatements = mutable.LinkedHashSet[String]() + protected val reusableOpenStatements = mutable.LinkedHashSet[String]() // set of close statements for RichFunction that will be added only once // we use a LinkedHashSet to keep the insertion order - private val reusableCloseStatements = mutable.LinkedHashSet[String]() + protected val reusableCloseStatements = mutable.LinkedHashSet[String]() // set of statements that will be added only once per record // we use a LinkedHashSet to keep the insertion order - private val reusablePerRecordStatements = mutable.LinkedHashSet[String]() + protected val reusablePerRecordStatements = mutable.LinkedHashSet[String]() // map of initial input unboxing expressions that will be added only once // (inputTerm, index) -> expr - private val reusableInputUnboxingExprs = mutable.Map[(String, Int), GeneratedExpression]() + protected val reusableInputUnboxingExprs = mutable.Map[(String, Int), GeneratedExpression]() // set of constructor statements that will be added only once // we use a LinkedHashSet to keep the insertion order - private val reusableConstructorStatements = mutable.LinkedHashSet[(String, String)]() + protected val reusableConstructorStatements = mutable.LinkedHashSet[(String, String)]() /** * @return code block of statements that need to be placed in the member area of the Function @@ -1458,9 +1459,10 @@ abstract class CodeGenerator( * Adds a reusable [[UserDefinedFunction]] to the member area of the generated [[Function]]. * * @param function [[UserDefinedFunction]] object to be instantiated during runtime + * @param contextTerm [[RuntimeContext]] term to access the [[RuntimeContext]] * @return member variable term */ - def addReusableFunction(function: UserDefinedFunction): String = { + def addReusableFunction(function: UserDefinedFunction, contextTerm: String = null): String = { val classQualifier = function.getClass.getCanonicalName val functionSerializedData = UserDefinedFunctionUtils.serialize(function) val fieldTerm = s"function_${function.functionIdentifier}" @@ -1480,10 +1482,15 @@ abstract class CodeGenerator( reusableInitStatements.add(functionDeserialization) - val openFunction = + val openFunction = if (contextTerm != null) { + s""" + |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}($contextTerm)); + """.stripMargin + } else { s""" |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}(getRuntimeContext())); """.stripMargin + } reusableOpenStatements.add(openFunction) val closeFunction = diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala new file mode 100644 index 0000000000000..a450c2ce1e552 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala @@ -0,0 +1,111 @@ +/* + * 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.dataview + +import org.apache.flink.api.common.typeutils._ +import org.apache.flink.api.common.typeutils.base.{CollectionSerializerConfigSnapshot, ListSerializer} +import org.apache.flink.core.memory.{DataInputView, DataOutputView} +import org.apache.flink.table.api.dataview.ListView + +/** + * A serializer for [[ListView]]. The serializer relies on an element + * serializer for the serialization of the list's elements. + * + * The serialization format for the list is as follows: four bytes for the length of the list, + * followed by the serialized representation of each element. + * + * @param listSerializer List serializer. + * @tparam T The type of element in the list. + */ +class ListViewSerializer[T](val listSerializer: ListSerializer[T]) + extends TypeSerializer[ListView[T]] { + + override def isImmutableType: Boolean = false + + override def duplicate(): TypeSerializer[ListView[T]] = { + new ListViewSerializer[T](listSerializer.duplicate().asInstanceOf[ListSerializer[T]]) + } + + override def createInstance(): ListView[T] = { + new ListView[T] + } + + override def copy(from: ListView[T]): ListView[T] = { + new ListView[T](null, listSerializer.copy(from.list)) + } + + override def copy(from: ListView[T], reuse: ListView[T]): ListView[T] = copy(from) + + override def getLength: Int = -1 + + override def serialize(record: ListView[T], target: DataOutputView): Unit = { + listSerializer.serialize(record.list, target) + } + + override def deserialize(source: DataInputView): ListView[T] = { + new ListView[T](null, listSerializer.deserialize(source)) + } + + override def deserialize(reuse: ListView[T], source: DataInputView): ListView[T] = + deserialize(source) + + override def copy(source: DataInputView, target: DataOutputView): Unit = + listSerializer.copy(source, target) + + override def canEqual(obj: scala.Any): Boolean = obj != null && obj.getClass == getClass + + override def hashCode(): Int = listSerializer.hashCode() + + override def equals(obj: Any): Boolean = canEqual(this) && + listSerializer.equals(obj.asInstanceOf[ListViewSerializer[_]].listSerializer) + + override def snapshotConfiguration(): TypeSerializerConfigSnapshot = + listSerializer.snapshotConfiguration() + + // copy and modified from ListSerializer.ensureCompatibility + override def ensureCompatibility( + configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[ListView[T]] = { + + configSnapshot match { + case snapshot: CollectionSerializerConfigSnapshot[_] => + val previousListSerializerAndConfig = snapshot.getSingleNestedSerializerAndConfig + + val compatResult = CompatibilityUtil.resolveCompatibilityResult( + previousListSerializerAndConfig.f0, + classOf[UnloadableDummyTypeSerializer[_]], + previousListSerializerAndConfig.f1, + listSerializer.getElementSerializer) + + if (!compatResult.isRequiresMigration) { + CompatibilityResult.compatible[ListView[T]] + } else if (compatResult.getConvertDeserializer != null) { + CompatibilityResult.requiresMigration( + new ListViewSerializer[T]( + new ListSerializer[T]( + new TypeDeserializerAdapter[T](compatResult.getConvertDeserializer)) + ) + ) + } else { + CompatibilityResult.requiresMigration[ListView[T]] + } + + case _ => CompatibilityResult.requiresMigration[ListView[T]] + } + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewTypeInfo.scala new file mode 100644 index 0000000000000..a10b6754a7c43 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewTypeInfo.scala @@ -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.table.dataview + +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.ListSerializer +import org.apache.flink.table.api.dataview.ListView + +/** + * [[ListView]] type information. + * + * @param elementType element type information + * @tparam T element type + */ +class ListViewTypeInfo[T](val elementType: TypeInformation[T]) + extends TypeInformation[ListView[T]] { + + override def isBasicType: Boolean = false + + override def isTupleType: Boolean = false + + override def getArity: Int = 1 + + override def getTotalFields: Int = 1 + + override def getTypeClass: Class[ListView[T]] = classOf[ListView[T]] + + override def isKeyType: Boolean = false + + override def createSerializer(config: ExecutionConfig): TypeSerializer[ListView[T]] = { + val typeSer = elementType.createSerializer(config) + new ListViewSerializer[T](new ListSerializer[T](typeSer)) + } + + override def canEqual(obj: scala.Any): Boolean = obj != null && obj.getClass == getClass + + override def hashCode(): Int = 31 * elementType.hashCode + + override def equals(obj: Any): Boolean = canEqual(obj) && { + obj match { + case other: ListViewTypeInfo[T] => + elementType.equals(other.elementType) + case _ => false + } + } + + override def toString: String = s"ListView<$elementType>" +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewTypeInfoFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewTypeInfoFactory.scala new file mode 100644 index 0000000000000..eda6cb9893f26 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewTypeInfoFactory.scala @@ -0,0 +1,43 @@ +/* + * 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.dataview + +import java.lang.reflect.Type +import java.util + +import org.apache.flink.api.common.typeinfo.{TypeInfoFactory, TypeInformation} +import org.apache.flink.api.java.typeutils.GenericTypeInfo +import org.apache.flink.table.api.dataview.ListView + +class ListViewTypeInfoFactory[T] extends TypeInfoFactory[ListView[T]] { + + override def createTypeInfo( + t: Type, + genericParameters: util.Map[String, TypeInformation[_]]): TypeInformation[ListView[T]] = { + + var elementType = genericParameters.get("T") + + if (elementType == null) { + // we might can get the elementType later from the ListView constructor + elementType = new GenericTypeInfo(classOf[Any]) + } + + new ListViewTypeInfo[T](elementType.asInstanceOf[TypeInformation[T]]) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala new file mode 100644 index 0000000000000..c53f10c37e597 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala @@ -0,0 +1,121 @@ +/* + * 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.dataview + +import org.apache.flink.api.common.typeutils._ +import org.apache.flink.api.common.typeutils.base.{MapSerializer, MapSerializerConfigSnapshot} +import org.apache.flink.core.memory.{DataInputView, DataOutputView} +import org.apache.flink.table.api.dataview.MapView + +/** + * A serializer for [[MapView]]. The serializer relies on a key serializer and a value + * serializer for the serialization of the map's key-value pairs. + * + * The serialization format for the map is as follows: four bytes for the length of the map, + * followed by the serialized representation of each key-value pair. To allow null values, + * each value is prefixed by a null marker. + * + * @param mapSerializer Map serializer. + * @tparam K The type of the keys in the map. + * @tparam V The type of the values in the map. + */ +class MapViewSerializer[K, V](val mapSerializer: MapSerializer[K, V]) + extends TypeSerializer[MapView[K, V]] { + + override def isImmutableType: Boolean = false + + override def duplicate(): TypeSerializer[MapView[K, V]] = + new MapViewSerializer[K, V]( + mapSerializer.duplicate().asInstanceOf[MapSerializer[K, V]]) + + override def createInstance(): MapView[K, V] = { + new MapView[K, V]() + } + + override def copy(from: MapView[K, V]): MapView[K, V] = { + new MapView[K, V](null, null, mapSerializer.copy(from.map)) + } + + override def copy(from: MapView[K, V], reuse: MapView[K, V]): MapView[K, V] = copy(from) + + override def getLength: Int = -1 // var length + + override def serialize(record: MapView[K, V], target: DataOutputView): Unit = { + mapSerializer.serialize(record.map, target) + } + + override def deserialize(source: DataInputView): MapView[K, V] = { + new MapView[K, V](null, null, mapSerializer.deserialize(source)) + } + + override def deserialize(reuse: MapView[K, V], source: DataInputView): MapView[K, V] = + deserialize(source) + + override def copy(source: DataInputView, target: DataOutputView): Unit = + mapSerializer.copy(source, target) + + override def canEqual(obj: Any): Boolean = obj != null && obj.getClass == getClass + + override def hashCode(): Int = mapSerializer.hashCode() + + override def equals(obj: Any): Boolean = canEqual(this) && + mapSerializer.equals(obj.asInstanceOf[MapViewSerializer[_, _]].mapSerializer) + + override def snapshotConfiguration(): TypeSerializerConfigSnapshot = + mapSerializer.snapshotConfiguration() + + // copy and modified from MapSerializer.ensureCompatibility + override def ensureCompatibility(configSnapshot: TypeSerializerConfigSnapshot) + : CompatibilityResult[MapView[K, V]] = { + + configSnapshot match { + case snapshot: MapSerializerConfigSnapshot[_, _] => + val previousKvSerializersAndConfigs = snapshot.getNestedSerializersAndConfigs + + val keyCompatResult = CompatibilityUtil.resolveCompatibilityResult( + previousKvSerializersAndConfigs.get(0).f0, + classOf[UnloadableDummyTypeSerializer[_]], + previousKvSerializersAndConfigs.get(0).f1, + mapSerializer.getKeySerializer) + + val valueCompatResult = CompatibilityUtil.resolveCompatibilityResult( + previousKvSerializersAndConfigs.get(1).f0, + classOf[UnloadableDummyTypeSerializer[_]], + previousKvSerializersAndConfigs.get(1).f1, + mapSerializer.getValueSerializer) + + if (!keyCompatResult.isRequiresMigration && !valueCompatResult.isRequiresMigration) { + CompatibilityResult.compatible[MapView[K, V]] + } else if (keyCompatResult.getConvertDeserializer != null + && valueCompatResult.getConvertDeserializer != null) { + CompatibilityResult.requiresMigration( + new MapViewSerializer[K, V]( + new MapSerializer[K, V]( + new TypeDeserializerAdapter[K](keyCompatResult.getConvertDeserializer), + new TypeDeserializerAdapter[V](valueCompatResult.getConvertDeserializer)) + ) + ) + } else { + CompatibilityResult.requiresMigration[MapView[K, V]] + } + + case _ => CompatibilityResult.requiresMigration[MapView[K, V]] + } + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewTypeInfo.scala new file mode 100644 index 0000000000000..ec5c2226e4278 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewTypeInfo.scala @@ -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.dataview + +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.MapSerializer +import org.apache.flink.table.api.dataview.MapView + +/** + * [[MapView]] type information. + * + * @param keyType key type information + * @param valueType value type information + * @tparam K key type + * @tparam V value type + */ +class MapViewTypeInfo[K, V]( + val keyType: TypeInformation[K], + val valueType: TypeInformation[V]) + extends TypeInformation[MapView[K, V]] { + + override def isBasicType = false + + override def isTupleType = false + + override def getArity = 1 + + override def getTotalFields = 1 + + override def getTypeClass: Class[MapView[K, V]] = classOf[MapView[K, V]] + + override def isKeyType: Boolean = false + + override def createSerializer(config: ExecutionConfig): TypeSerializer[MapView[K, V]] = { + val keySer = keyType.createSerializer(config) + val valueSer = valueType.createSerializer(config) + new MapViewSerializer[K, V](new MapSerializer[K, V](keySer, valueSer)) + } + + override def canEqual(obj: scala.Any): Boolean = obj != null && obj.getClass == getClass + + override def hashCode(): Int = 31 * keyType.hashCode + valueType.hashCode + + override def equals(obj: Any): Boolean = canEqual(obj) && { + obj match { + case other: MapViewTypeInfo[_, _] => + keyType.equals(other.keyType) && + valueType.equals(other.valueType) + case _ => false + } + } + + override def toString: String = s"MapView<$keyType, $valueType>" +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewTypeInfoFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewTypeInfoFactory.scala new file mode 100644 index 0000000000000..33c3ffe2a2e38 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewTypeInfoFactory.scala @@ -0,0 +1,51 @@ +/* + * 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.dataview + +import java.lang.reflect.Type +import java.util + +import org.apache.flink.api.common.typeinfo.{TypeInfoFactory, TypeInformation} +import org.apache.flink.api.java.typeutils.GenericTypeInfo +import org.apache.flink.table.api.dataview.MapView + +class MapViewTypeInfoFactory[K, V] extends TypeInfoFactory[MapView[K, V]] { + + override def createTypeInfo( + t: Type, + genericParameters: util.Map[String, TypeInformation[_]]): TypeInformation[MapView[K, V]] = { + + var keyType = genericParameters.get("K") + var valueType = genericParameters.get("V") + + if (keyType == null) { + // we might can get the keyType later from the MapView constructor + keyType = new GenericTypeInfo(classOf[Any]) + } + + if (valueType == null) { + // we might can get the valueType later from the MapView constructor + valueType = new GenericTypeInfo(classOf[Any]) + } + + new MapViewTypeInfo[K, V]( + keyType.asInstanceOf[TypeInformation[K]], + valueType.asInstanceOf[TypeInformation[V]]) + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/StateListView.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/StateListView.scala new file mode 100644 index 0000000000000..70756ca761414 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/StateListView.scala @@ -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.table.dataview + +import java.util +import java.lang.{Iterable => JIterable} + +import org.apache.flink.api.common.state._ +import org.apache.flink.table.api.dataview.ListView + +/** + * [[ListView]] use state backend. + * + * @param state list state + * @tparam T element type + */ +class StateListView[T](state: ListState[T]) extends ListView[T] { + + override def get: JIterable[T] = state.get() + + override def add(value: T): Unit = state.add(value) + + override def addAll(list: util.List[T]): Unit = { + val iterator = list.iterator() + while (iterator.hasNext) { + state.add(iterator.next()) + } + } + + override def clear(): Unit = state.clear() +} + diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/StateMapView.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/StateMapView.scala new file mode 100644 index 0000000000000..22f5f0b23459f --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/StateMapView.scala @@ -0,0 +1,54 @@ +/* + * 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.dataview + +import java.util +import java.lang.{Iterable => JIterable} + +import org.apache.flink.api.common.state.MapState +import org.apache.flink.table.api.dataview.MapView + +/** + * [[MapView]] use state backend. + * + * @param state map state + * @tparam K key type + * @tparam V value type + */ +class StateMapView[K, V](state: MapState[K, V]) extends MapView[K, V] { + + override def get(key: K): V = state.get(key) + + override def put(key: K, value: V): Unit = state.put(key, value) + + override def putAll(map: util.Map[K, V]): Unit = state.putAll(map) + + override def remove(key: K): Unit = state.remove(key) + + override def contains(key: K): Boolean = state.contains(key) + + override def entries: JIterable[util.Map.Entry[K, V]] = state.entries() + + override def keys: JIterable[K] = state.keys() + + override def values: JIterable[V] = state.values() + + override def iterator: util.Iterator[util.Map.Entry[K, V]] = state.iterator() + + override def clear(): Unit = state.clear() +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala index b44c28ea34039..f53bcdeca7ae3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala @@ -19,6 +19,7 @@ package org.apache.flink.table.functions.utils +import java.util import java.lang.{Integer => JInt, Long => JLong} import java.lang.reflect.{Method, Modifier} import java.sql.{Date, Time, Timestamp} @@ -29,7 +30,10 @@ import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.{SqlCallBinding, SqlFunction} import org.apache.flink.api.common.functions.InvalidTypesException import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.TypeExtractor +import org.apache.flink.api.common.typeutils.CompositeType +import org.apache.flink.api.java.typeutils.{PojoField, PojoTypeInfo, TypeExtractor} +import org.apache.flink.table.api.dataview._ +import org.apache.flink.table.dataview._ import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.api.{TableEnvironment, TableException, ValidationException} import org.apache.flink.table.expressions._ @@ -38,6 +42,8 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl import org.apache.flink.table.plan.schema.FlinkTableFunctionImpl import org.apache.flink.util.InstantiationUtil +import scala.collection.mutable + object UserDefinedFunctionUtils { /** @@ -306,6 +312,111 @@ object UserDefinedFunctionUtils { // Utilities for user-defined functions // ---------------------------------------------------------------------------------------------- + /** + * Remove StateView fields from accumulator type information. + * + * @param index index of aggregate function + * @param aggFun aggregate function + * @param accType accumulator type information, only support pojo type + * @param isStateBackedDataViews is data views use state backend + * @return mapping of accumulator type information and data view config which contains id, + * field name and state descriptor + */ + def removeStateViewFieldsFromAccTypeInfo( + index: Int, + aggFun: AggregateFunction[_, _], + accType: TypeInformation[_], + isStateBackedDataViews: Boolean) + : (TypeInformation[_], Option[Seq[DataViewSpec[_]]]) = { + + /** Recursively checks if composite type includes a data view type. */ + def includesDataView(ct: CompositeType[_]): Boolean = { + (0 until ct.getArity).exists(i => + ct.getTypeAt(i) match { + case nestedCT: CompositeType[_] => includesDataView(nestedCT) + case t: TypeInformation[_] if t.getTypeClass == classOf[ListView[_]] => true + case t: TypeInformation[_] if t.getTypeClass == classOf[MapView[_, _]] => true + case _ => false + } + ) + } + + val acc = aggFun.createAccumulator() + accType match { + case pojoType: PojoTypeInfo[_] if pojoType.getArity > 0 => + val arity = pojoType.getArity + val newPojoFields = new util.ArrayList[PojoField]() + val accumulatorSpecs = new mutable.ArrayBuffer[DataViewSpec[_]] + for (i <- 0 until arity) { + val pojoField = pojoType.getPojoFieldAt(i) + val field = pojoField.getField + val fieldName = field.getName + field.setAccessible(true) + + pojoField.getTypeInformation match { + case ct: CompositeType[_] if includesDataView(ct) => + throw new TableException( + "MapView and ListView only supported at first level of accumulators of Pojo type.") + case map: MapViewTypeInfo[_, _] => + val mapView = field.get(acc).asInstanceOf[MapView[_, _]] + if (mapView != null) { + val keyTypeInfo = mapView.keyTypeInfo + val valueTypeInfo = mapView.valueTypeInfo + val newTypeInfo = if (keyTypeInfo != null && valueTypeInfo != null) { + new MapViewTypeInfo(keyTypeInfo, valueTypeInfo) + } else { + map + } + + // create map view specs with unique id (used as state name) + var spec = MapViewSpec( + "agg" + index + "$" + fieldName, + field, + newTypeInfo) + + accumulatorSpecs += spec + if (!isStateBackedDataViews) { + // add data view field if it is not backed by a state backend. + // data view fields which are backed by state backend are not serialized. + newPojoFields.add(new PojoField(field, newTypeInfo)) + } + } + + case list: ListViewTypeInfo[_] => + val listView = field.get(acc).asInstanceOf[ListView[_]] + if (listView != null) { + val elementTypeInfo = listView.elementTypeInfo + val newTypeInfo = if (elementTypeInfo != null) { + new ListViewTypeInfo(elementTypeInfo) + } else { + list + } + + // create list view specs with unique is (used as state name) + var spec = ListViewSpec( + "agg" + index + "$" + fieldName, + field, + newTypeInfo) + + accumulatorSpecs += spec + if (!isStateBackedDataViews) { + // add data view field if it is not backed by a state backend. + // data view fields which are backed by state backend are not serialized. + newPojoFields.add(new PojoField(field, newTypeInfo)) + } + } + + case _ => newPojoFields.add(pojoField) + } + } + (new PojoTypeInfo(accType.getTypeClass, newPojoFields), Some(accumulatorSpecs)) + case ct: CompositeType[_] if includesDataView(ct) => + throw new TableException( + "MapView and ListView only supported in accumulators of POJO type.") + case _ => (accType, None) + } + } + /** * Tries to infer the TypeInformation of an AggregateFunction's return type. * 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 6304dc4bb69f8..58940d06abb05 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,6 +32,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo 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.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.calcite.FlinkTypeFactory @@ -82,11 +83,12 @@ object AggregateUtil { isRowsClause: Boolean) : ProcessFunction[CRow, CRow] = { - val (aggFields, aggregates, accTypes) = + val (aggFields, aggregates, accTypes, accSpecs) = transformToAggregateFunctions( namedAggregates.map(_.getKey), inputType, - needRetraction = false) + needRetraction = false, + isStateBackedDataViews = true) val aggregationStateType: RowTypeInfo = new RowTypeInfo(accTypes: _*) @@ -107,7 +109,8 @@ object AggregateUtil { outputArity, needRetract = false, needMerge = false, - needReset = false + needReset = false, + accConfig = Some(accSpecs) ) if (rowTimeIdx.isDefined) { @@ -160,11 +163,12 @@ object AggregateUtil { generateRetraction: Boolean, consumeRetraction: Boolean): ProcessFunction[CRow, CRow] = { - val (aggFields, aggregates, accTypes) = + val (aggFields, aggregates, accTypes, accSpecs) = transformToAggregateFunctions( namedAggregates.map(_.getKey), inputRowType, - consumeRetraction) + consumeRetraction, + isStateBackedDataViews = true) val aggMapping = aggregates.indices.map(_ + groupings.length).toArray @@ -185,7 +189,8 @@ object AggregateUtil { outputArity, consumeRetraction, needMerge = false, - needReset = false + needReset = false, + accConfig = Some(accSpecs) ) new GroupAggProcessFunction( @@ -223,11 +228,12 @@ object AggregateUtil { : ProcessFunction[CRow, CRow] = { val needRetract = true - val (aggFields, aggregates, accTypes) = + val (aggFields, aggregates, accTypes, accSpecs) = transformToAggregateFunctions( namedAggregates.map(_.getKey), inputType, - needRetract) + needRetract, + isStateBackedDataViews = true) val aggregationStateType: RowTypeInfo = new RowTypeInfo(accTypes: _*) val inputRowType = CRowTypeInfo(inputTypeInfo) @@ -249,7 +255,8 @@ object AggregateUtil { outputArity, needRetract, needMerge = false, - needReset = true + needReset = false, + accConfig = Some(accSpecs) ) if (rowTimeIdx.isDefined) { @@ -323,7 +330,7 @@ object AggregateUtil { : MapFunction[Row, Row] = { val needRetract = false - val (aggFieldIndexes, aggregates, accTypes) = transformToAggregateFunctions( + val (aggFieldIndexes, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), inputType, needRetract) @@ -380,7 +387,8 @@ object AggregateUtil { outputArity, needRetract, needMerge = false, - needReset = true + needReset = true, + None ) new DataSetWindowAggMapFunction( @@ -428,7 +436,7 @@ object AggregateUtil { : RichGroupReduceFunction[Row, Row] = { val needRetract = false - val (aggFieldIndexes, aggregates, accTypes) = transformToAggregateFunctions( + val (aggFieldIndexes, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), physicalInputRowType, needRetract) @@ -458,7 +466,8 @@ object AggregateUtil { keysAndAggregatesArity + 1, needRetract, needMerge = true, - needReset = true + needReset = true, + None ) new DataSetSlideTimeWindowAggReduceGroupFunction( genFunction, @@ -541,7 +550,7 @@ object AggregateUtil { : RichGroupReduceFunction[Row, Row] = { val needRetract = false - val (aggFieldIndexes, aggregates, _) = transformToAggregateFunctions( + val (aggFieldIndexes, aggregates, _, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), physicalInputRowType, needRetract) @@ -561,7 +570,8 @@ object AggregateUtil { outputType.getFieldCount, needRetract, needMerge = true, - needReset = true + needReset = true, + None ) val genFinalAggFunction = generator.generateAggregations( @@ -577,7 +587,8 @@ object AggregateUtil { outputType.getFieldCount, needRetract, needMerge = true, - needReset = true + needReset = true, + None ) val keysAndAggregatesArity = groupings.length + namedAggregates.length @@ -686,7 +697,7 @@ object AggregateUtil { groupings: Array[Int]): MapPartitionFunction[Row, Row] = { val needRetract = false - val (aggFieldIndexes, aggregates, accTypes) = transformToAggregateFunctions( + val (aggFieldIndexes, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), physicalInputRowType, needRetract) @@ -718,7 +729,8 @@ object AggregateUtil { groupings.length + aggregates.length + 2, needRetract, needMerge = true, - needReset = true + needReset = true, + None ) new DataSetSessionWindowAggregatePreProcessor( @@ -759,7 +771,7 @@ object AggregateUtil { : GroupCombineFunction[Row, Row] = { val needRetract = false - val (aggFieldIndexes, aggregates, accTypes) = transformToAggregateFunctions( + val (aggFieldIndexes, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), physicalInputRowType, needRetract) @@ -792,7 +804,8 @@ object AggregateUtil { groupings.length + aggregates.length + 2, needRetract, needMerge = true, - needReset = true + needReset = true, + None ) new DataSetSessionWindowAggregatePreProcessor( @@ -825,7 +838,7 @@ object AggregateUtil { RichGroupReduceFunction[Row, Row]) = { val needRetract = false - val (aggInFields, aggregates, accTypes) = transformToAggregateFunctions( + val (aggInFields, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), inputType, needRetract) @@ -872,7 +885,8 @@ object AggregateUtil { groupings.length + aggregates.length, needRetract, needMerge = false, - needReset = true + needReset = true, + None ) // compute mapping of forwarded grouping keys @@ -898,7 +912,8 @@ object AggregateUtil { outputType.getFieldCount, needRetract, needMerge = true, - needReset = true + needReset = true, + None ) ( @@ -921,7 +936,8 @@ object AggregateUtil { outputType.getFieldCount, needRetract, needMerge = false, - needReset = true + needReset = true, + None ) ( @@ -996,7 +1012,7 @@ object AggregateUtil { : (DataStreamAggFunction[CRow, Row, Row], RowTypeInfo, RowTypeInfo) = { val needRetract = false - val (aggFields, aggregates, accTypes) = + val (aggFields, aggregates, accTypes, _) = transformToAggregateFunctions( namedAggregates.map(_.getKey), inputType, @@ -1018,7 +1034,8 @@ object AggregateUtil { outputArity, needRetract, needMerge, - needReset = false + needReset = false, + None ) val aggResultTypes = namedAggregates.map(a => FlinkTypeFactory.toTypeInfo(a.left.getType)) @@ -1159,8 +1176,12 @@ object AggregateUtil { private def transformToAggregateFunctions( aggregateCalls: Seq[AggregateCall], inputType: RelDataType, - needRetraction: Boolean) - : (Array[Array[Int]], Array[TableAggregateFunction[_, _]], Array[TypeInformation[_]]) = { + needRetraction: Boolean, + isStateBackedDataViews: Boolean = false) + : (Array[Array[Int]], + Array[TableAggregateFunction[_, _]], + Array[TypeInformation[_]], + Array[Seq[DataViewSpec[_]]]) = { // store the aggregate fields of each aggregate function, by the same order of aggregates. val aggFieldIndexes = new Array[Array[Int]](aggregateCalls.size) @@ -1398,14 +1419,28 @@ object AggregateUtil { } } + val accSpecs = new Array[Seq[DataViewSpec[_]]](aggregateCalls.size) + // create accumulator type information for every aggregate function aggregates.zipWithIndex.foreach { case (agg, index) => - if (null == accTypes(index)) { + if (accTypes(index) != null) { + val (accType, specs) = removeStateViewFieldsFromAccTypeInfo(index, + agg, + accTypes(index), + isStateBackedDataViews) + if (specs.isDefined) { + accSpecs(index) = specs.get + accTypes(index) = accType + } else { + accSpecs(index) = Seq() + } + } else { + accSpecs(index) = Seq() accTypes(index) = getAccumulatorTypeOfAggregateFunction(agg) } } - (aggFieldIndexes, aggregates, accTypes) + (aggFieldIndexes, aggregates, accTypes, accSpecs) } private def createRowTypeForKeysAndAggregates( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GeneratedAggregations.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GeneratedAggregations.scala index 5f48e091996e5..7b201142ce4fa 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GeneratedAggregations.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GeneratedAggregations.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.runtime.aggregate -import org.apache.flink.api.common.functions.Function +import org.apache.flink.api.common.functions.{Function, RuntimeContext} import org.apache.flink.types.Row /** @@ -26,6 +26,14 @@ import org.apache.flink.types.Row */ abstract class GeneratedAggregations extends Function { + /** + * Setup method for [[org.apache.flink.table.functions.AggregateFunction]]. + * It can be used for initialization work. By default, this method does nothing. + * + * @param ctx The runtime context. + */ + def open(ctx: RuntimeContext) + /** * Sets the results of the aggregations (partial or final) to the output row. * Final results are computed with the aggregation function. @@ -100,6 +108,17 @@ abstract class GeneratedAggregations extends Function { * aggregated results */ def resetAccumulator(accumulators: Row) + + /** + * Cleanup for the accumulators. + */ + def cleanup() + + /** + * Tear-down method for [[org.apache.flink.table.functions.AggregateFunction]]. + * It can be used for clean up work. By default, this method does nothing. + */ + def close() } class SingleElementIterable[T] extends java.lang.Iterable[T] { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index 690a7c3d6cda9..a4769875997f9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -23,9 +23,8 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.common.state.{StateDescriptor, ValueState, ValueStateDescriptor} import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.api.common.state.ValueState import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.slf4j.{Logger, LoggerFactory} @@ -65,6 +64,7 @@ class GroupAggProcessFunction( genAggregations.code) LOG.debug("Instantiating AggregateHelper.") function = clazz.newInstance() + function.open(getRuntimeContext) newRow = new CRow(function.createOutputRow(), true) prevRow = new CRow(function.createOutputRow(), false) @@ -162,7 +162,11 @@ class GroupAggProcessFunction( if (needToCleanupState(timestamp)) { cleanupState(state, cntState) + function.cleanup() } } + override def close(): Unit = { + function.close() + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala index ab3dc1db63248..5c2851904f5b6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala @@ -22,10 +22,7 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.apache.flink.api.common.state.ValueState -import org.apache.flink.api.common.state.ValueStateDescriptor -import org.apache.flink.api.common.state.MapState -import org.apache.flink.api.common.state.MapStateDescriptor +import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ListTypeInfo import java.util.{ArrayList, List => JList} @@ -71,6 +68,8 @@ class ProcTimeBoundedRangeOver( genAggregations.code) LOG.debug("Instantiating AggregateHelper.") function = clazz.newInstance() + function.open(getRuntimeContext) + output = new CRow(function.createOutputRow(), true) // We keep the elements received in a MapState indexed based on their ingestion time @@ -121,6 +120,7 @@ class ProcTimeBoundedRangeOver( if (needToCleanupState(timestamp)) { // clean up and return cleanupState(rowMapState, accumulatorState) + function.cleanup() return } @@ -201,4 +201,7 @@ class ProcTimeBoundedRangeOver( accumulatorState.update(accumulators) } + override def close(): Unit = { + function.close() + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala index ccc4b461e291e..09d98adac8fff 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala @@ -23,11 +23,8 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.types.Row import org.apache.flink.util.{Collector, Preconditions} -import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.common.state._ import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.api.common.state.ValueState -import org.apache.flink.api.common.state.MapState -import org.apache.flink.api.common.state.MapStateDescriptor import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ListTypeInfo import java.util.{List => JList} @@ -75,6 +72,7 @@ class ProcTimeBoundedRowsOver( genAggregations.code) LOG.debug("Instantiating AggregateHelper.") function = clazz.newInstance() + function.open(getRuntimeContext) output = new CRow(function.createOutputRow(), true) // We keep the elements received in a Map state keyed @@ -194,6 +192,11 @@ class ProcTimeBoundedRowsOver( if (needToCleanupState(timestamp)) { cleanupState(rowMapState, accumulatorState, counterState, smallestTsState) + function.cleanup() } } + + override def close(): Unit = { + function.close() + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala index 7a7b44d378b28..4fb559544d5db 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala @@ -21,9 +21,8 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.apache.flink.api.common.state.ValueStateDescriptor +import org.apache.flink.api.common.state.{StateDescriptor, ValueState, ValueStateDescriptor} import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.api.common.state.ValueState import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.CRow @@ -56,6 +55,7 @@ class ProcTimeUnboundedOver( genAggregations.code) LOG.debug("Instantiating AggregateHelper.") function = clazz.newInstance() + function.open(getRuntimeContext) output = new CRow(function.createOutputRow(), true) val stateDescriptor: ValueStateDescriptor[Row] = @@ -97,6 +97,11 @@ class ProcTimeUnboundedOver( if (needToCleanupState(timestamp)) { cleanupState(state) + function.cleanup() } } + + override def close(): Unit = { + function.close() + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala index 8a0d682379636..1ee269326ef55 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala @@ -78,6 +78,7 @@ class RowTimeBoundedRangeOver( genAggregations.code) LOG.debug("Instantiating AggregateHelper.") function = clazz.newInstance() + function.open(getRuntimeContext) output = new CRow(function.createOutputRow(), true) @@ -158,6 +159,7 @@ class RowTimeBoundedRangeOver( if (noRecordsToProcess) { // we clean the state cleanupState(dataState, accumulatorState, lastTriggeringTsState) + function.cleanup() } else { // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. @@ -242,6 +244,10 @@ class RowTimeBoundedRangeOver( // update cleanup timer registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } + + override def close(): Unit = { + function.close() + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala index ba6584664daa0..60200bcc1df9a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala @@ -84,6 +84,7 @@ class RowTimeBoundedRowsOver( genAggregations.code) LOG.debug("Instantiating AggregateHelper.") function = clazz.newInstance() + function.open(getRuntimeContext) output = new CRow(function.createOutputRow(), true) @@ -168,6 +169,7 @@ class RowTimeBoundedRowsOver( if (noRecordsToProcess) { // We clean the state cleanupState(dataState, accumulatorState, dataCountState, lastTriggeringTsState) + function.cleanup() } else { // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. @@ -264,6 +266,10 @@ class RowTimeBoundedRowsOver( // update cleanup timer registerProcessingCleanupTimer(ctx, ctx.timerService().currentProcessingTime()) } + + override def close(): Unit = { + function.close() + } } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala index 9210c002c6b52..c8183c944b851 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala @@ -71,6 +71,7 @@ abstract class RowTimeUnboundedOver( genAggregations.code) LOG.debug("Instantiating AggregateHelper.") function = clazz.newInstance() + function.open(getRuntimeContext) output = new CRow(function.createOutputRow(), true) sortedTimestamps = new util.LinkedList[Long]() @@ -150,6 +151,7 @@ abstract class RowTimeUnboundedOver( if (noRecordsToProcess) { // we clean the state cleanupState(rowMapState, accumulatorState) + function.cleanup() } else { // There are records left to process because a watermark has not been received yet. // This would only happen if the input stream has stopped. So we don't need to clean up. @@ -241,6 +243,9 @@ abstract class RowTimeUnboundedOver( lastAccumulator: Row, out: Collector[CRow]): Unit + override def close(): Unit = { + function.close() + } } /** diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedAggFunctions.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedAggFunctions.java index 4d06bc2c88eec..14f812aee4f32 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedAggFunctions.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/runtime/utils/JavaUserDefinedAggFunctions.java @@ -18,7 +18,10 @@ package org.apache.flink.table.runtime.utils; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.dataview.ListView; +import org.apache.flink.table.api.dataview.MapView; import org.apache.flink.table.functions.AggregateFunction; import java.util.Iterator; @@ -135,4 +138,200 @@ public void retract(WeightedAvgAccum accumulator, int iValue, int iWeight) { accumulator.count -= iWeight; } } + + /** + * CountDistinct accumulator. + */ + public static class CountDistinctAccum { + public MapView map; + public long count; + } + + /** + * CountDistinct aggregate. + */ + public static class CountDistinct extends AggregateFunction { + + @Override + public CountDistinctAccum createAccumulator() { + CountDistinctAccum accum = new CountDistinctAccum(); + accum.map = new MapView<>(Types.STRING, Types.INT); + accum.count = 0L; + return accum; + } + + //Overloaded accumulate method + public void accumulate(CountDistinctAccum accumulator, String id) { + try { + Integer cnt = accumulator.map.get(id); + if (cnt != null) { + cnt += 1; + accumulator.map.put(id, cnt); + } else { + accumulator.map.put(id, 1); + accumulator.count += 1; + } + } catch (Exception e) { + e.printStackTrace(); + } + } + + //Overloaded accumulate method + public void accumulate(CountDistinctAccum accumulator, long id) { + try { + Integer cnt = accumulator.map.get(String.valueOf(id)); + if (cnt != null) { + cnt += 1; + accumulator.map.put(String.valueOf(id), cnt); + } else { + accumulator.map.put(String.valueOf(id), 1); + accumulator.count += 1; + } + } catch (Exception e) { + e.printStackTrace(); + } + } + + @Override + public Long getValue(CountDistinctAccum accumulator) { + return accumulator.count; + } + } + + /** + * CountDistinct aggregate with merge. + */ + public static class CountDistinctWithMerge extends CountDistinct { + + //Overloaded merge method + public void merge(CountDistinctAccum acc, Iterable it) { + Iterator iter = it.iterator(); + while (iter.hasNext()) { + CountDistinctAccum mergeAcc = iter.next(); + acc.count += mergeAcc.count; + + try { + Iterator itrMap = mergeAcc.map.keys().iterator(); + while (itrMap.hasNext()) { + String key = itrMap.next(); + Integer cnt = mergeAcc.map.get(key); + if (acc.map.contains(key)) { + acc.map.put(key, acc.map.get(key) + cnt); + } else { + acc.map.put(key, cnt); + } + } + } catch (Exception e) { + e.printStackTrace(); + } + } + } + } + + /** + * CountDistinct aggregate with merge and reset. + */ + public static class CountDistinctWithMergeAndReset extends CountDistinctWithMerge { + + //Overloaded retract method + public void resetAccumulator(CountDistinctAccum acc) { + acc.map.clear(); + acc.count = 0; + } + } + + /** + * CountDistinct aggregate with retract. + */ + public static class CountDistinctWithRetractAndReset extends CountDistinct { + + //Overloaded retract method + public void retract(CountDistinctAccum accumulator, long id) { + try { + Integer cnt = accumulator.map.get(String.valueOf(id)); + if (cnt != null) { + cnt -= 1; + if (cnt <= 0) { + accumulator.map.remove(String.valueOf(id)); + accumulator.count -= 1; + } else { + accumulator.map.put(String.valueOf(id), cnt); + } + } + } catch (Exception e) { + e.printStackTrace(); + } + } + + //Overloaded retract method + public void resetAccumulator(CountDistinctAccum acc) { + acc.map.clear(); + acc.count = 0; + } + } + + /** + * Accumulator for test DataView. + */ + public static class DataViewTestAccum { + public MapView map; + public MapView map2; // for test not initialized + public long count; + private ListView list = new ListView<>(Types.LONG); + + public ListView getList() { + return list; + } + + public void setList(ListView list) { + this.list = list; + } + } + + public static boolean isCloseCalled = false; + + /** + * Aggregate for test DataView. + */ + public static class DataViewTestAgg extends AggregateFunction { + @Override + public DataViewTestAccum createAccumulator() { + DataViewTestAccum accum = new DataViewTestAccum(); + accum.map = new MapView<>(Types.STRING, Types.INT); + accum.count = 0L; + return accum; + } + + // Overloaded accumulate method + public void accumulate(DataViewTestAccum accumulator, String a, long b) { + try { + if (!accumulator.map.contains(a)) { + accumulator.map.put(a, 1); + accumulator.count++; + } + + accumulator.list.add(b); + } catch (Exception e) { + e.printStackTrace(); + } + } + + @Override + public Long getValue(DataViewTestAccum accumulator) { + long sum = accumulator.count; + try { + for (Long value : accumulator.list.get()) { + sum += value; + } + } catch (Exception e) { + e.printStackTrace(); + } + return sum; + } + + @Override + public void close() { + isCloseCalled = true; + } + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/dataview/ListViewSerializerTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/dataview/ListViewSerializerTest.scala new file mode 100644 index 0000000000000..3f70bcef21e0e --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/dataview/ListViewSerializerTest.scala @@ -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.table.dataview + +import java.lang.Long +import java.util.Random + +import org.apache.flink.api.common.typeutils.base.{ListSerializer, LongSerializer} +import org.apache.flink.api.common.typeutils.{SerializerTestBase, TypeSerializer} +import org.apache.flink.table.api.dataview.ListView + +/** + * A test for the [[ListViewSerializer]]. + */ +class ListViewSerializerTest extends SerializerTestBase[ListView[Long]] { + + override protected def createSerializer(): TypeSerializer[ListView[Long]] = { + val listSerializer = new ListSerializer[Long](LongSerializer.INSTANCE) + new ListViewSerializer[Long](listSerializer) + } + + override protected def getLength: Int = -1 + + override protected def getTypeClass: Class[ListView[Long]] = classOf[ListView[Long]] + + override protected def getTestData: Array[ListView[Long]] = { + val rnd = new Random(321332) + + // empty + val listview1 = new ListView[Long]() + + // single element + val listview2 = new ListView[Long]() + listview2.add(12345L) + + // multiple elements + val listview3 = new ListView[Long]() + var i = 0 + while (i < rnd.nextInt(200)) { + listview3.add(rnd.nextLong) + i += 1 + } + + Array[ListView[Long]](listview1, listview2, listview3) + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/dataview/MapViewSerializerTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/dataview/MapViewSerializerTest.scala new file mode 100644 index 0000000000000..15f9b0240b282 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/dataview/MapViewSerializerTest.scala @@ -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.table.dataview + +import java.lang.Long +import java.util.Random + +import org.apache.flink.api.common.typeutils.base.{LongSerializer, MapSerializer, StringSerializer} +import org.apache.flink.api.common.typeutils.{SerializerTestBase, TypeSerializer} +import org.apache.flink.table.api.dataview.MapView + +/** + * A test for the [[MapViewSerializer]]. + */ +class MapViewSerializerTest extends SerializerTestBase[MapView[Long, String]] { + + override protected def createSerializer(): TypeSerializer[MapView[Long, String]] = { + val mapSerializer = new MapSerializer[Long, String](LongSerializer.INSTANCE, + StringSerializer.INSTANCE) + new MapViewSerializer[Long, String](mapSerializer) + } + + override protected def getLength: Int = -1 + + override protected def getTypeClass: Class[MapView[Long, String]] = + classOf[MapView[Long, String]] + + override protected def getTestData: Array[MapView[Long, String]] = { + val rnd = new Random(321654) + + // empty + val mapview1 = new MapView[Long, String]() + + // single element + val mapview2 = new MapView[Long, String]() + mapview2.put(12345L, "12345L") + + // multiple elements + val mapview3 = new MapView[Long, String]() + var i = 0 + while (i < rnd.nextInt(200)) { + mapview3.put(rnd.nextLong, Long.toString(rnd.nextLong)) + i += 1 + } + + // null-value maps + val mapview4 = new MapView[Long, String]() + mapview4.put(999L, null) + + Array[MapView[Long, String]](mapview1, mapview2, mapview3, mapview4) + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala index d563f9636ce3f..cf96d19966b6a 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/AggregateITCase.scala @@ -23,7 +23,7 @@ import java.math.BigDecimal import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.table.api.TableEnvironment -import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMergeAndReset +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinctWithMergeAndReset, WeightedAvgWithMergeAndReset} import org.apache.flink.table.api.scala._ import org.apache.flink.table.functions.aggfunctions.CountAggFunction import org.apache.flink.table.runtime.utils.TableProgramsCollectionTestBase @@ -226,13 +226,14 @@ class AggregationsITCase( val tEnv = TableEnvironment.getTableEnvironment(env, config) val countFun = new CountAggFunction val wAvgFun = new WeightedAvgWithMergeAndReset + val countDistinct = new CountDistinctWithMergeAndReset val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c) .groupBy('b) - .select('b, 'a.sum, countFun('c), wAvgFun('b, 'a), wAvgFun('a, 'a)) + .select('b, 'a.sum, countFun('c), wAvgFun('b, 'a), wAvgFun('a, 'a), countDistinct('c)) - val expected = "1,1,1,1,1\n" + "2,5,2,2,2\n" + "3,15,3,3,5\n" + "4,34,4,4,8\n" + - "5,65,5,5,13\n" + "6,111,6,6,18\n" + val expected = "1,1,1,1,1,1\n" + "2,5,2,2,2,2\n" + "3,15,3,3,5,3\n" + "4,34,4,4,8,4\n" + + "5,65,5,5,13,5\n" + "6,111,6,6,18,6\n" val results = t.toDataSet[Row].collect() TestBaseUtils.compareResultAsText(results.asJava, expected) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala index 04aada6ca738f..67164b71b54c4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/harness/HarnessTestBase.scala @@ -170,6 +170,14 @@ class HarnessTestBase { | return new org.apache.flink.types.Row(5); | } | + | public void open(org.apache.flink.api.common.functions.RuntimeContext ctx) { + | } + | + | public void cleanup() { + | } + | + | public void close() { + | } |/******* This test does not use the following methods *******/ | public org.apache.flink.types.Row mergeAccumulatorsPair( | org.apache.flink.types.Row a, @@ -282,6 +290,15 @@ class HarnessTestBase { | public final void resetAccumulator( | org.apache.flink.types.Row accs) { | } + | + | public void open(org.apache.flink.api.common.functions.RuntimeContext ctx) { + | } + | + | public void cleanup() { + | } + | + | public void close() { + | } |} |""".stripMargin diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala index 744ac4602daba..eb3d37fb0375f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala @@ -24,7 +24,8 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api.scala._ import org.apache.flink.table.runtime.utils.StreamITCase.RetractingSink import org.apache.flink.table.api.{StreamQueryConfig, TableEnvironment} -import org.apache.flink.table.runtime.utils.{StreamITCase, StreamTestData, StreamingWithStateTestBase} +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, DataViewTestAgg} +import org.apache.flink.table.runtime.utils.{JavaUserDefinedAggFunctions, StreamITCase, StreamTestData, StreamingWithStateTestBase} import org.apache.flink.types.Row import org.junit.Assert.assertEquals import org.junit.Test @@ -154,4 +155,42 @@ class AggregateITCase extends StreamingWithStateTestBase { "12,3,5,1", "5,3,4,2") assertEquals(expected.sorted, StreamITCase.retractedResults.sorted) } + + @Test + def testGroupAggregateWithStateBackend(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + env.setStateBackend(getStateBackend) + val tEnv = TableEnvironment.getTableEnvironment(env) + StreamITCase.clear + + val data = new mutable.MutableList[(Int, Long, String)] + data.+=((1, 1L, "A")) + data.+=((2, 2L, "B")) + data.+=((3, 2L, "B")) + data.+=((4, 3L, "C")) + data.+=((5, 3L, "C")) + data.+=((6, 3L, "C")) + data.+=((7, 4L, "B")) + data.+=((8, 4L, "A")) + data.+=((9, 4L, "D")) + data.+=((10, 4L, "E")) + data.+=((11, 5L, "A")) + data.+=((12, 5L, "B")) + + val distinct = new CountDistinct + val testAgg = new DataViewTestAgg + val t = env.fromCollection(data).toTable(tEnv, 'a, 'b, 'c) + .groupBy('b) + .select('b, distinct('c), testAgg('c, 'b)) + + val results = t.toRetractStream[Row](queryConfig) + results.addSink(new StreamITCase.RetractingSink) + env.execute() + + val expected = List("1,1,2", "2,1,5", "3,1,10", "4,4,20", "5,2,12") + assertEquals(expected.sorted, StreamITCase.retractedResults.sorted) + + // verify agg close is called + assert(JavaUserDefinedAggFunctions.isCloseCalled) + } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/GroupWindowITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/GroupWindowITCase.scala index 1561da0417edb..f6e739efc8958 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/GroupWindowITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/GroupWindowITCase.scala @@ -29,7 +29,7 @@ import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase -import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{WeightedAvg, WeightedAvgWithMerge} +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, CountDistinctWithMerge, WeightedAvg, WeightedAvgWithMerge} import org.apache.flink.table.functions.aggfunctions.CountAggFunction import org.apache.flink.table.runtime.stream.table.GroupWindowITCase._ import org.apache.flink.table.runtime.utils.StreamITCase @@ -75,19 +75,21 @@ class GroupWindowITCase extends StreamingMultipleProgramsTestBase { val countFun = new CountAggFunction val weightAvgFun = new WeightedAvg + val countDistinct = new CountDistinct val windowedTable = table .window(Slide over 2.rows every 1.rows on 'proctime as 'w) .groupBy('w, 'string) .select('string, countFun('int), 'int.avg, - weightAvgFun('long, 'int), weightAvgFun('int, 'int)) + weightAvgFun('long, 'int), weightAvgFun('int, 'int), + countDistinct('long)) val results = windowedTable.toAppendStream[Row](queryConfig) results.addSink(new StreamITCase.StringSink[Row]) env.execute() - val expected = Seq("Hello world,1,3,8,3", "Hello world,2,3,12,3", "Hello,1,2,2,2", - "Hello,2,2,3,2", "Hi,1,1,1,1") + val expected = Seq("Hello world,1,3,8,3,1", "Hello world,2,3,12,3,2", "Hello,1,2,2,2,1", + "Hello,2,2,3,2,2", "Hi,1,1,1,1,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @@ -112,6 +114,7 @@ class GroupWindowITCase extends StreamingMultipleProgramsTestBase { val countFun = new CountAggFunction val weightAvgFun = new WeightedAvgWithMerge + val countDistinct = new CountDistinctWithMerge val stream = env .fromCollection(sessionWindowTestdata) @@ -122,13 +125,14 @@ class GroupWindowITCase extends StreamingMultipleProgramsTestBase { .window(Session withGap 5.milli on 'rowtime as 'w) .groupBy('w, 'string) .select('string, countFun('int), 'int.avg, - weightAvgFun('long, 'int), weightAvgFun('int, 'int)) + weightAvgFun('long, 'int), weightAvgFun('int, 'int), + countDistinct('long)) val results = windowedTable.toAppendStream[Row] results.addSink(new StreamITCase.StringSink[Row]) env.execute() - val expected = Seq("Hello World,1,9,9,9", "Hello,1,16,16,16", "Hello,4,3,5,5") + val expected = Seq("Hello World,1,9,9,9,1", "Hello,1,16,16,16,1", "Hello,4,3,5,5,4") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @@ -143,18 +147,21 @@ class GroupWindowITCase extends StreamingMultipleProgramsTestBase { val table = stream.toTable(tEnv, 'long, 'int, 'string, 'proctime.proctime) val countFun = new CountAggFunction val weightAvgFun = new WeightedAvg + val countDistinct = new CountDistinct val windowedTable = table .window(Tumble over 2.rows on 'proctime as 'w) .groupBy('w) .select(countFun('string), 'int.avg, - weightAvgFun('long, 'int), weightAvgFun('int, 'int)) + weightAvgFun('long, 'int), weightAvgFun('int, 'int), + countDistinct('long) + ) val results = windowedTable.toAppendStream[Row](queryConfig) results.addSink(new StreamITCase.StringSink[Row]) env.execute() - val expected = Seq("2,1,1,1", "2,2,6,2") + val expected = Seq("2,1,1,1,2", "2,2,6,2,2") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @@ -171,22 +178,24 @@ class GroupWindowITCase extends StreamingMultipleProgramsTestBase { val table = stream.toTable(tEnv, 'long, 'int, 'string, 'rowtime.rowtime) val countFun = new CountAggFunction val weightAvgFun = new WeightedAvg + val countDistinct = new CountDistinct val windowedTable = table .window(Tumble over 5.milli on 'rowtime as 'w) .groupBy('w, 'string) .select('string, countFun('string), 'int.avg, weightAvgFun('long, 'int), - weightAvgFun('int, 'int), 'int.min, 'int.max, 'int.sum, 'w.start, 'w.end) + weightAvgFun('int, 'int), 'int.min, 'int.max, 'int.sum, 'w.start, 'w.end, + countDistinct('long)) val results = windowedTable.toAppendStream[Row] results.addSink(new StreamITCase.StringSink[Row]) env.execute() val expected = Seq( - "Hello world,1,3,8,3,3,3,3,1970-01-01 00:00:00.005,1970-01-01 00:00:00.01", - "Hello world,1,3,16,3,3,3,3,1970-01-01 00:00:00.015,1970-01-01 00:00:00.02", - "Hello,2,2,3,2,2,2,4,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005", - "Hi,1,1,1,1,1,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005") + "Hello world,1,3,8,3,3,3,3,1970-01-01 00:00:00.005,1970-01-01 00:00:00.01,1", + "Hello world,1,3,16,3,3,3,3,1970-01-01 00:00:00.015,1970-01-01 00:00:00.02,1", + "Hello,2,2,3,2,2,2,4,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005,2", + "Hi,1,1,1,1,1,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @@ -208,17 +217,18 @@ class GroupWindowITCase extends StreamingMultipleProgramsTestBase { val table = stream.toTable(tEnv, 'long, 'int, 'string, 'int2, 'int3, 'proctime.proctime) val weightAvgFun = new WeightedAvg + val countDistinct = new CountDistinct val windowedTable = table .window(Slide over 2.rows every 1.rows on 'proctime as 'w) .groupBy('w, 'int2, 'int3, 'string) - .select(weightAvgFun('long, 'int)) + .select(weightAvgFun('long, 'int), countDistinct('long)) val results = windowedTable.toAppendStream[Row] results.addSink(new StreamITCase.StringSink[Row]) env.execute() - val expected = Seq("12", "8", "2", "3", "1") + val expected = Seq("12,2", "8,1", "2,1", "3,2", "1,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/OverWindowITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/OverWindowITCase.scala index 73484d2551db8..54971b2bc81fd 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/OverWindowITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/OverWindowITCase.scala @@ -25,7 +25,7 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceCont import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.table.api.TableEnvironment -import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvg +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.{CountDistinct, CountDistinctWithRetractAndReset, WeightedAvg} import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc0 import org.apache.flink.table.api.scala._ import org.apache.flink.table.functions.aggfunctions.CountAggFunction @@ -51,6 +51,7 @@ class OverWindowITCase extends StreamingWithStateTestBase { (6L, 6, "Hello"), (7L, 7, "Hello World"), (8L, 8, "Hello World"), + (8L, 8, "Hello World"), (20L, 20, "Hello World")) val env = StreamExecutionEnvironment.getExecutionEnvironment @@ -62,20 +63,24 @@ class OverWindowITCase extends StreamingWithStateTestBase { val table = stream.toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime) val countFun = new CountAggFunction val weightAvgFun = new WeightedAvg + val countDist = new CountDistinct val windowedTable = table .window( Over partitionBy 'c orderBy 'proctime preceding UNBOUNDED_ROW as 'w) - .select('c, countFun('b) over 'w as 'mycount, weightAvgFun('a, 'b) over 'w as 'wAvg) - .select('c, 'mycount, 'wAvg) + .select('c, + countFun('b) over 'w as 'mycount, + weightAvgFun('a, 'b) over 'w as 'wAvg, + countDist('a) over 'w as 'countDist) + .select('c, 'mycount, 'wAvg, 'countDist) val results = windowedTable.toAppendStream[Row] results.addSink(new StreamITCase.StringSink[Row]) env.execute() val expected = Seq( - "Hello World,1,7", "Hello World,2,7", "Hello World,3,14", - "Hello,1,1", "Hello,2,1", "Hello,3,2", "Hello,4,3", "Hello,5,3", "Hello,6,4") + "Hello World,1,7,1", "Hello World,2,7,2", "Hello World,3,7,2", "Hello World,4,13,3", + "Hello,1,1,1", "Hello,2,1,2", "Hello,3,2,3", "Hello,4,3,4", "Hello,5,3,5", "Hello,6,4,6") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @@ -112,6 +117,7 @@ class OverWindowITCase extends StreamingWithStateTestBase { val countFun = new CountAggFunction val weightAvgFun = new WeightedAvg val plusOne = new JavaFunc0 + val countDist = new CountDistinct val windowedTable = table .window(Over partitionBy 'a orderBy 'rowtime preceding UNBOUNDED_RANGE following @@ -128,26 +134,27 @@ class OverWindowITCase extends StreamingWithStateTestBase { 'b.max over 'w, 'b.min over 'w, ('b.min over 'w).abs(), - weightAvgFun('b, 'a) over 'w) + weightAvgFun('b, 'a) over 'w, + countDist('c) over 'w as 'countDist) val result = windowedTable.toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) env.execute() val expected = mutable.MutableList( - "1,1,Hello,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2", - "1,2,Hello,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2", - "1,3,Hello world,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2", - "1,1,Hi,7,SUM:7,4,5,5,[1, 3],1,3,1,1,1", - "2,1,Hello,1,SUM:1,1,2,2,[1, 1],1,1,1,1,1", - "2,2,Hello world,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2", - "2,3,Hello world,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2", - "1,4,Hello world,11,SUM:11,5,6,6,[2, 4],2,4,1,1,2", - "1,5,Hello world,29,SUM:29,8,9,9,[3, 7],3,7,1,1,3", - "1,6,Hello world,29,SUM:29,8,9,9,[3, 7],3,7,1,1,3", - "1,7,Hello world,29,SUM:29,8,9,9,[3, 7],3,7,1,1,3", - "2,4,Hello world,15,SUM:15,5,6,6,[3, 5],3,5,1,1,3", - "2,5,Hello world,15,SUM:15,5,6,6,[3, 5],3,5,1,1,3" + "1,1,Hello,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2,2", + "1,2,Hello,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2,2", + "1,3,Hello world,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2,2", + "1,1,Hi,7,SUM:7,4,5,5,[1, 3],1,3,1,1,1,3", + "2,1,Hello,1,SUM:1,1,2,2,[1, 1],1,1,1,1,1,1", + "2,2,Hello world,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2,2", + "2,3,Hello world,6,SUM:6,3,4,4,[2, 3],2,3,1,1,2,2", + "1,4,Hello world,11,SUM:11,5,6,6,[2, 4],2,4,1,1,2,3", + "1,5,Hello world,29,SUM:29,8,9,9,[3, 7],3,7,1,1,3,3", + "1,6,Hello world,29,SUM:29,8,9,9,[3, 7],3,7,1,1,3,3", + "1,7,Hello world,29,SUM:29,8,9,9,[3, 7],3,7,1,1,3,3", + "2,4,Hello world,15,SUM:15,5,6,6,[3, 5],3,5,1,1,3,2", + "2,5,Hello world,15,SUM:15,5,6,6,[3, 5],3,5,1,1,3,2" ) assertEquals(expected.sorted, StreamITCase.testResults.sorted) @@ -179,32 +186,33 @@ class OverWindowITCase extends StreamingWithStateTestBase { env.setParallelism(1) StreamITCase.testResults = mutable.MutableList() + val countDist = new CountDistinctWithRetractAndReset val stream = env.fromCollection(data) val table = stream.toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'proctime.proctime) val windowedTable = table .window(Over partitionBy 'a orderBy 'proctime preceding 4.rows following CURRENT_ROW as 'w) - .select('a, 'c.sum over 'w, 'c.min over 'w) + .select('a, 'c.sum over 'w, 'c.min over 'w, countDist('e) over 'w) val result = windowedTable.toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) env.execute() val expected = mutable.MutableList( - "1,0,0", - "2,1,1", - "2,3,1", - "3,3,3", - "3,7,3", - "3,12,3", - "4,6,6", - "4,13,6", - "4,21,6", - "4,30,6", - "5,10,10", - "5,21,10", - "5,33,10", - "5,46,10", - "5,60,10") + "1,0,0,1", + "2,1,1,1", + "2,3,1,2", + "3,3,3,1", + "3,7,3,1", + "3,12,3,2", + "4,6,6,1", + "4,13,6,2", + "4,21,6,2", + "4,30,6,2", + "5,10,10,1", + "5,21,10,2", + "5,33,10,2", + "5,46,10,3", + "5,60,10,3") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @@ -240,25 +248,27 @@ class OverWindowITCase extends StreamingWithStateTestBase { val tEnv = TableEnvironment.getTableEnvironment(env) StreamITCase.clear + val countDist = new CountDistinctWithRetractAndReset val table = env.addSource[(Long, Int, String)]( new RowTimeSourceFunction[(Long, Int, String)](data)) .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime) val windowedTable = table .window(Over partitionBy 'c orderBy 'rowtime preceding 2.rows following CURRENT_ROW as 'w) - .select('c, 'a, 'a.count over 'w, 'a.sum over 'w) + .select('c, 'a, 'a.count over 'w, 'a.sum over 'w, countDist('a) over 'w) val result = windowedTable.toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) env.execute() val expected = mutable.MutableList( - "Hello,1,1,1", "Hello,1,2,2", "Hello,1,3,3", - "Hello,2,3,4", "Hello,2,3,5", "Hello,2,3,6", - "Hello,3,3,7", "Hello,4,3,9", "Hello,5,3,12", - "Hello,6,3,15", - "Hello World,7,1,7", "Hello World,7,2,14", "Hello World,7,3,21", - "Hello World,7,3,21", "Hello World,8,3,22", "Hello World,20,3,35") + "Hello,1,1,1,1", "Hello,1,2,2,1", "Hello,1,3,3,1", + "Hello,2,3,4,2", "Hello,2,3,5,2", "Hello,2,3,6,1", + "Hello,3,3,7,2", "Hello,4,3,9,3", "Hello,5,3,12,3", + "Hello,6,3,15,3", + "Hello World,7,1,7,1", "Hello World,7,2,14,1", "Hello World,7,3,21,1", + "Hello World,7,3,21,1", "Hello World,8,3,22,2", "Hello World,20,3,35,3") + assertEquals(expected.sorted, StreamITCase.testResults.sorted) } @@ -302,6 +312,7 @@ class OverWindowITCase extends StreamingWithStateTestBase { val tEnv = TableEnvironment.getTableEnvironment(env) StreamITCase.clear + val countDist = new CountDistinctWithRetractAndReset val table = env.addSource[(Long, Int, String)]( new RowTimeSourceFunction[(Long, Int, String)](data)) .toTable(tEnv, 'a, 'b, 'c, 'rowtime.rowtime) @@ -309,23 +320,24 @@ class OverWindowITCase extends StreamingWithStateTestBase { val windowedTable = table .window( Over partitionBy 'c orderBy 'rowtime preceding 1.seconds following CURRENT_RANGE as 'w) - .select('c, 'b, 'a.count over 'w, 'a.sum over 'w) + .select('c, 'b, 'a.count over 'w, 'a.sum over 'w, countDist('a) over 'w) val result = windowedTable.toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) env.execute() val expected = mutable.MutableList( - "Hello,1,1,1", "Hello,15,2,2", "Hello,16,3,3", - "Hello,2,6,9", "Hello,3,6,9", "Hello,2,6,9", - "Hello,3,4,9", - "Hello,4,2,7", - "Hello,5,2,9", - "Hello,6,2,11", "Hello,65,2,12", - "Hello,9,2,12", "Hello,9,2,12", "Hello,18,3,18", - "Hello World,7,1,7", "Hello World,17,3,21", "Hello World,77,3,21", "Hello World,18,1,7", - "Hello World,8,2,15", - "Hello World,20,1,20") + "Hello,1,1,1,1", "Hello,15,2,2,1", "Hello,16,3,3,1", + "Hello,2,6,9,2", "Hello,3,6,9,2", "Hello,2,6,9,2", + "Hello,3,4,9,2", + "Hello,4,2,7,2", + "Hello,5,2,9,2", + "Hello,6,2,11,2", "Hello,65,2,12,1", + "Hello,9,2,12,1", "Hello,9,2,12,1", "Hello,18,3,18,1", + "Hello World,7,1,7,1", "Hello World,17,3,21,1", + "Hello World,77,3,21,1", "Hello World,18,1,7,1", + "Hello World,8,2,15,2", + "Hello World,20,1,20,1") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } } From 68fdaa57e35b8ee30a262aad4d26926b18054c57 Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Wed, 9 Aug 2017 20:54:16 +0800 Subject: [PATCH 065/129] [FLINK-7245] [table] Add operators to hold back watermarks with static delays. This closes #4530. --- ...dCoProcessOperatorWithWatermarkDelay.scala | 58 ++++++++++++ ...yedProcessOperatorWithWatermarkDelay.scala | 59 ++++++++++++ ...rocessOperatorWithWatermarkDelayTest.scala | 93 +++++++++++++++++++ ...rocessOperatorWithWatermarkDelayTest.scala | 78 ++++++++++++++++ .../api/operators/AbstractStreamOperator.java | 2 +- .../operators/InternalTimeServiceManager.java | 2 +- 6 files changed, 290 insertions(+), 2 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelayTest.scala create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelayTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala new file mode 100644 index 0000000000000..f25de256e96ac --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelay.scala @@ -0,0 +1,58 @@ +/* + * 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.runtime.operators + +import org.apache.flink.streaming.api.functions.co.CoProcessFunction +import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator +import org.apache.flink.streaming.api.watermark.Watermark + +/** + * A [[KeyedCoProcessOperator]] that supports holding back watermarks with a static delay. + */ +class KeyedCoProcessOperatorWithWatermarkDelay[KEY, IN1, IN2, OUT]( + private val flatMapper: CoProcessFunction[IN1, IN2, OUT], + private val watermarkDelay: Long = 0L) + extends KeyedCoProcessOperator[KEY, IN1, IN2, OUT](flatMapper) { + + /** emits watermark without delay */ + def emitWithoutDelay(mark: Watermark): Unit = output.emitWatermark(mark) + + /** emits watermark with delay */ + def emitWithDelay(mark: Watermark): Unit = { + output.emitWatermark(new Watermark(mark.getTimestamp - watermarkDelay)) + } + + if (watermarkDelay < 0) { + throw new IllegalArgumentException("The watermark delay should be non-negative.") + } + + // choose watermark emitter + val emitter: Watermark => Unit = if (watermarkDelay == 0) { + emitWithoutDelay + } else { + emitWithDelay + } + + @throws[Exception] + override def processWatermark(mark: Watermark) { + if (timeServiceManager != null) timeServiceManager.advanceWatermark(mark) + + emitter(mark) + } + +} 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 new file mode 100644 index 0000000000000..74b4773005a65 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelay.scala @@ -0,0 +1,59 @@ +/* + * 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.runtime.operators + +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.operators.KeyedProcessOperator +import org.apache.flink.streaming.api.watermark.Watermark + +/** + * A [[KeyedProcessOperator]] 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) { + + /** emits watermark without delay */ + def emitWithoutDelay(mark: Watermark): Unit = output.emitWatermark(mark) + + /** emits watermark with delay */ + def emitWithDelay(mark: Watermark): Unit = { + output.emitWatermark(new Watermark(mark.getTimestamp - watermarkDelay)) + } + + if (watermarkDelay < 0) { + throw new IllegalArgumentException("The watermark delay should be non-negative.") + } + + // choose watermark emitter + val emitter: Watermark => Unit = if (watermarkDelay == 0) { + emitWithoutDelay + } else { + emitWithDelay + } + + @throws[Exception] + override def processWatermark(mark: Watermark) { + if (timeServiceManager != null) timeServiceManager.advanceWatermark(mark) + + emitter(mark) + } + +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelayTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelayTest.scala new file mode 100644 index 0000000000000..243a034268af0 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedCoProcessOperatorWithWatermarkDelayTest.scala @@ -0,0 +1,93 @@ +/* + * 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.runtime.operators + +import java.util.concurrent.ConcurrentLinkedQueue + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.streaming.api.functions.co.CoProcessFunction +import org.apache.flink.streaming.api.watermark.Watermark +import org.apache.flink.streaming.util.{KeyedTwoInputStreamOperatorTestHarness, TestHarnessUtil} +import org.apache.flink.util.{Collector, TestLogger} +import org.junit.Test + +/** + * Tests [[KeyedCoProcessOperatorWithWatermarkDelay]]. + */ +class KeyedCoProcessOperatorWithWatermarkDelayTest extends TestLogger { + + @Test + def testHoldingBackWatermarks(): Unit = { + val operator = new KeyedCoProcessOperatorWithWatermarkDelay[String, Integer, String, String]( + new EmptyCoProcessFunction, 100) + val testHarness = new KeyedTwoInputStreamOperatorTestHarness[String, Integer, String, String]( + operator, + new IntToStringKeySelector, new CoIdentityKeySelector[String], + BasicTypeInfo.STRING_TYPE_INFO) + + testHarness.setup() + testHarness.open() + testHarness.processWatermark1(new Watermark(101)) + testHarness.processWatermark2(new Watermark(202)) + testHarness.processWatermark1(new Watermark(103)) + testHarness.processWatermark2(new Watermark(204)) + + val expectedOutput = new ConcurrentLinkedQueue[AnyRef] + expectedOutput.add(new Watermark(1)) + expectedOutput.add(new Watermark(3)) + + TestHarnessUtil.assertOutputEquals( + "Output was not correct.", + expectedOutput, + testHarness.getOutput) + + testHarness.close() + } + + @Test(expected = classOf[IllegalArgumentException]) + def testDelayParameter(): Unit = { + new KeyedCoProcessOperatorWithWatermarkDelay[AnyRef, Integer, String, String]( + new EmptyCoProcessFunction, -1) + } +} + +private class EmptyCoProcessFunction extends CoProcessFunction[Integer, String, String] { + override def processElement1( + value: Integer, + ctx: CoProcessFunction[Integer, String, String]#Context, + out: Collector[String]): Unit = { + // do nothing + } + + override def processElement2( + value: String, + ctx: CoProcessFunction[Integer, String, String]#Context, + out: Collector[String]): Unit = { + //do nothing + } +} + + +private class IntToStringKeySelector extends KeySelector[Integer, String] { + override def getKey(value: Integer): String = String.valueOf(value) +} + +private class CoIdentityKeySelector[T] extends KeySelector[T, T] { + override def getKey(value: T): T = value +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelayTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelayTest.scala new file mode 100644 index 0000000000000..d419453726570 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/operators/KeyedProcessOperatorWithWatermarkDelayTest.scala @@ -0,0 +1,78 @@ +/* + * 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.runtime.operators + +import java.util.concurrent.ConcurrentLinkedQueue + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.streaming.api.watermark.Watermark +import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, TestHarnessUtil} +import org.apache.flink.util.{Collector, TestLogger} +import org.junit.Test + +/** + * Tests [[KeyedProcessOperatorWithWatermarkDelay]]. + */ +class KeyedProcessOperatorWithWatermarkDelayTest extends TestLogger { + + @Test + def testHoldingBackWatermarks(): Unit = { + val operator = new KeyedProcessOperatorWithWatermarkDelay[Integer, Integer, String]( + new EmptyProcessFunction, 100) + val testHarness = new KeyedOneInputStreamOperatorTestHarness[Integer, Integer, String]( + operator, new IdentityKeySelector, BasicTypeInfo.INT_TYPE_INFO) + + testHarness.setup() + testHarness.open() + testHarness.processWatermark(new Watermark(101)) + testHarness.processWatermark(new Watermark(103)) + + val expectedOutput = new ConcurrentLinkedQueue[AnyRef] + expectedOutput.add(new Watermark(1)) + expectedOutput.add(new Watermark(3)) + + TestHarnessUtil.assertOutputEquals( + "Output was not correct.", + expectedOutput, + testHarness.getOutput) + + testHarness.close() + } + + @Test(expected = classOf[IllegalArgumentException]) + def testDelayParameter(): Unit = { + new KeyedProcessOperatorWithWatermarkDelay[Integer, Integer, String]( + new EmptyProcessFunction, -1) + } +} + +private class EmptyProcessFunction extends ProcessFunction[Integer, String] { + override def processElement( + value: Integer, + ctx: ProcessFunction[Integer, String]#Context, + out: Collector[String]): Unit = { + // do nothing + } +} + +private class IdentityKeySelector[T] extends KeySelector[T, T] { + override def getKey(value: T): T = value +} 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 a28fc304af0fb..fc043a8045d5c 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 @@ -160,7 +160,7 @@ public abstract class AbstractStreamOperator // ---------------- time handler ------------------ - private transient InternalTimeServiceManager timeServiceManager; + protected transient InternalTimeServiceManager timeServiceManager; // ---------------- two-input operator watermarks ------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java index 17af3aa4a3f18..7d5cb9188ec3e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimeServiceManager.java @@ -44,7 +44,7 @@ * @param The type of namespace used for the timers. */ @Internal -class InternalTimeServiceManager { +public class InternalTimeServiceManager { private final int totalKeyGroups; private final KeyGroupsList localKeyGroupRange; From 29e849b1bf9180a5aa5f2d500efb283a39839caa Mon Sep 17 00:00:00 2001 From: Yestin <873915231@qq.com> Date: Fri, 4 Aug 2017 17:21:03 +0800 Subject: [PATCH 066/129] [FLINK-7309] [table] Fix NullPointerException when selecting null fields. This closes #4479. --- .../flink/table/codegen/CodeGenerator.scala | 6 ++-- .../table/expressions/TemporalTypesTest.scala | 32 +++++++++++++++++-- 2 files changed, 33 insertions(+), 5 deletions(-) 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 154e8ad2341ad..bf6ee217b9683 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 @@ -1211,15 +1211,15 @@ abstract class CodeGenerator( } val wrappedCode = if (nullCheck && !isReference(fieldType)) { + // assumes that fieldType is a boxed primitive. s""" - |$tmpTypeTerm $tmpTerm = $unboxedFieldCode; - |boolean $nullTerm = $tmpTerm == null; + |boolean $nullTerm = $fieldTerm == null; |$resultTypeTerm $resultTerm; |if ($nullTerm) { | $resultTerm = $defaultValue; |} |else { - | $resultTerm = $tmpTerm; + | $resultTerm = $fieldTerm; |} |""".stripMargin } else if (nullCheck) { diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala index 1d761c36b68d8..8fae11a8f4de5 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala @@ -538,10 +538,31 @@ class TemporalTypesTest extends ExpressionTestBase { "1990-09-12 10:20:45.123") } + @Test + def testSelectNullValues(): Unit ={ + testAllApis( + 'f11, + "f11", + "f11", + "null" + ) + testAllApis( + 'f12, + "f12", + "f12", + "null" + ) + testAllApis( + 'f13, + "f13", + "f13", + "null" + ) + } // ---------------------------------------------------------------------------------------------- def testData: Row = { - val testData = new Row(11) + val testData = new Row(14) testData.setField(0, Date.valueOf("1990-10-14")) testData.setField(1, Time.valueOf("10:20:45")) testData.setField(2, Timestamp.valueOf("1990-10-14 10:20:45.123")) @@ -553,6 +574,10 @@ class TemporalTypesTest extends ExpressionTestBase { testData.setField(8, 1467012213000L) testData.setField(9, 24) testData.setField(10, 12000L) + // null selection test. + testData.setField(11, null) + testData.setField(12, null) + testData.setField(13, null) testData } @@ -568,6 +593,9 @@ class TemporalTypesTest extends ExpressionTestBase { Types.INT, Types.LONG, Types.INTERVAL_MONTHS, - Types.INTERVAL_MILLIS).asInstanceOf[TypeInformation[Any]] + Types.INTERVAL_MILLIS, + Types.SQL_DATE, + Types.SQL_TIME, + Types.SQL_TIMESTAMP).asInstanceOf[TypeInformation[Any]] } } From df7452d9811b0aa88919d7e3c1f6c34b36ac9b38 Mon Sep 17 00:00:00 2001 From: Haohui Mai Date: Wed, 23 Aug 2017 14:13:07 -0700 Subject: [PATCH 067/129] [FLINK-7398] [table] Add Logging trait to prevent serialization of Logger. This closes #4576. --- .../table/catalog/ExternalCatalogSchema.scala | 8 ++---- .../catalog/ExternalTableSourceUtil.scala | 6 ++-- .../datastream/DataStreamGroupAggregate.scala | 11 ++++---- .../DataStreamGroupWindowAggregate.scala | 10 +++---- .../datastream/DataStreamOverAggregate.scala | 16 +++++------ .../runtime/CRowCorrelateProcessRunner.scala | 7 ++--- .../flink/table/runtime/CRowMapRunner.scala | 7 ++--- .../runtime/CRowOutputProcessRunner.scala | 7 ++--- .../table/runtime/CRowProcessRunner.scala | 7 ++--- .../runtime/CorrelateFlatMapRunner.scala | 7 ++--- .../flink/table/runtime/FlatJoinRunner.scala | 9 +++--- .../flink/table/runtime/FlatMapRunner.scala | 9 +++--- .../flink/table/runtime/MapRunner.scala | 9 +++--- .../table/runtime/MapSideJoinRunner.scala | 9 +++--- .../aggregate/AggregateAggFunction.scala | 5 ++-- .../aggregate/DataSetAggFunction.scala | 5 ++-- .../aggregate/DataSetFinalAggFunction.scala | 5 ++-- .../aggregate/DataSetPreAggFunction.scala | 6 ++-- ...tSessionWindowAggReduceGroupFunction.scala | 8 +++--- ...etSessionWindowAggregatePreProcessor.scala | 8 +++--- ...lideTimeWindowAggReduceGroupFunction.scala | 6 ++-- ...SetSlideWindowAggReduceGroupFunction.scala | 6 ++-- ...bleCountWindowAggReduceGroupFunction.scala | 6 ++-- ...mbleTimeWindowAggReduceGroupFunction.scala | 6 ++-- .../DataSetWindowAggMapFunction.scala | 6 ++-- .../aggregate/GroupAggProcessFunction.scala | 14 +++++----- .../aggregate/ProcTimeBoundedRangeOver.scala | 6 ++-- .../aggregate/ProcTimeBoundedRowsOver.scala | 21 ++++++-------- .../aggregate/ProcTimeUnboundedOver.scala | 14 +++++----- .../aggregate/RowTimeBoundedRangeOver.scala | 6 ++-- .../aggregate/RowTimeBoundedRowsOver.scala | 7 ++--- .../aggregate/RowTimeUnboundedOver.scala | 7 ++--- .../conversion/CRowToJavaTupleMapRunner.scala | 7 ++--- .../CRowToScalaTupleMapRunner.scala | 7 ++--- .../runtime/io/CRowValuesInputFormat.scala | 7 ++--- .../table/runtime/io/ValuesInputFormat.scala | 9 +++--- .../join/ProcTimeWindowInnerJoin.scala | 7 ++--- .../org/apache/flink/table/util/Logging.scala | 28 +++++++++++++++++++ 38 files changed, 165 insertions(+), 164 deletions(-) create mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/table/util/Logging.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala index 197449ca5e6ff..c74066f022e20 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalogSchema.scala @@ -18,12 +18,12 @@ package org.apache.flink.table.catalog -import java.util.{Collections => JCollections, Collection => JCollection, LinkedHashSet => JLinkedHashSet, Set => JSet} +import java.util.{Collection => JCollection, Collections => JCollections, LinkedHashSet => JLinkedHashSet, Set => JSet} import org.apache.calcite.linq4j.tree.Expression import org.apache.calcite.schema._ import org.apache.flink.table.api.{CatalogNotExistException, TableNotExistException} -import org.slf4j.{Logger, LoggerFactory} +import org.apache.flink.table.util.Logging import scala.collection.JavaConverters._ @@ -38,9 +38,7 @@ import scala.collection.JavaConverters._ */ class ExternalCatalogSchema( catalogIdentifier: String, - catalog: ExternalCatalog) extends Schema { - - private val LOG: Logger = LoggerFactory.getLogger(this.getClass) + catalog: ExternalCatalog) extends Schema with Logging { /** * Looks up a sub-schema by the given sub-schema name in the external catalog. 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 ccc2e9ebd4d8c..6bacac1fd50cb 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 @@ -27,9 +27,9 @@ import org.apache.flink.table.api.{AmbiguousTableSourceConverterException, NoMat import org.apache.flink.table.plan.schema.{StreamTableSourceTable, TableSourceTable} import org.apache.flink.table.plan.stats.FlinkStatistic import org.apache.flink.table.sources.{StreamTableSource, TableSource} +import org.apache.flink.table.util.Logging import org.apache.flink.util.InstantiationUtil import org.reflections.Reflections -import org.slf4j.{Logger, LoggerFactory} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -37,13 +37,11 @@ import scala.collection.mutable /** * The utility class is used to convert ExternalCatalogTable to TableSourceTable. */ -object ExternalTableSourceUtil { +object ExternalTableSourceUtil extends Logging { // config file to specify scan package to search TableSourceConverter private val tableSourceConverterConfigFileName = "tableSourceConverter.properties" - private val LOG: Logger = LoggerFactory.getLogger(this.getClass) - // registered table type with the TableSourceConverter. // Key is table type name, Value is set of converter class. private val tableTypeToTableSourceConvertersClazz = { 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 590d9be43bfd5..58c9d820ee59a 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 @@ -24,13 +24,13 @@ import org.apache.flink.api.java.functions.NullByteKeySelector import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment} import org.apache.flink.table.codegen.AggregationCodeGenerator -import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.plan.nodes.CommonAggregate -import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules +import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair +import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.slf4j.LoggerFactory +import org.apache.flink.table.util.Logging /** * @@ -55,9 +55,8 @@ class DataStreamGroupAggregate( groupings: Array[Int]) extends SingleRel(cluster, traitSet, inputNode) with CommonAggregate - with DataStreamRel { - - private val LOG = LoggerFactory.getLogger(this.getClass) + with DataStreamRel + with Logging { override def deriveRowType() = schema.relDataType 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 0cf86f7069bcd..b15350f9cfb3a 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 @@ -29,7 +29,6 @@ import org.apache.flink.streaming.api.windowing.triggers.PurgingTrigger import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow} import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty -import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.AggregationCodeGenerator import org.apache.flink.table.expressions.ExpressionUtils._ import org.apache.flink.table.expressions.ResolvedFieldReference @@ -44,7 +43,7 @@ import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval import org.apache.flink.table.runtime.triggers.StateCleaningCountTrigger -import org.slf4j.LoggerFactory +import org.apache.flink.table.util.Logging class DataStreamGroupWindowAggregate( window: LogicalWindow, @@ -56,9 +55,10 @@ class DataStreamGroupWindowAggregate( schema: RowSchema, inputSchema: RowSchema, grouping: Array[Int]) - extends SingleRel(cluster, traitSet, inputNode) with CommonAggregate with DataStreamRel { - - private val LOG = LoggerFactory.getLogger(this.getClass) + extends SingleRel(cluster, traitSet, inputNode) + with CommonAggregate + with DataStreamRel + with Logging { override def deriveRowType(): RelDataType = schema.relDataType 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 7bf342a8985d1..62345252c44ee 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 @@ -20,23 +20,23 @@ package org.apache.flink.table.plan.nodes.datastream import java.util.{List => JList} import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.Window.Group import org.apache.calcite.rel.core.{AggregateCall, Window} import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} -import org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING +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.calcite.FlinkTypeFactory -import org.apache.flink.table.plan.nodes.OverAggregate -import org.apache.flink.table.plan.schema.RowSchema -import org.apache.flink.table.runtime.aggregate._ -import org.apache.flink.api.java.functions.NullByteKeySelector import org.apache.flink.table.codegen.AggregationCodeGenerator +import org.apache.flink.table.plan.nodes.OverAggregate import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules +import org.apache.flink.table.plan.schema.RowSchema import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair +import org.apache.flink.table.runtime.aggregate._ import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.slf4j.LoggerFactory +import org.apache.flink.table.util.Logging class DataStreamOverAggregate( logicWindow: Window, @@ -47,8 +47,8 @@ class DataStreamOverAggregate( inputSchema: RowSchema) extends SingleRel(cluster, traitSet, inputNode) with OverAggregate - with DataStreamRel { - private val LOG = LoggerFactory.getLogger(this.getClass) + with DataStreamRel + with Logging { override def deriveRowType(): RelDataType = schema.relDataType diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala index 4f0a78550794a..2553d9cd67b1a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowCorrelateProcessRunner.scala @@ -25,9 +25,9 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.codegen.Compiler import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.{Logger, LoggerFactory} /** * A CorrelateProcessRunner with [[CRow]] input and [[CRow]] output. @@ -40,9 +40,8 @@ class CRowCorrelateProcessRunner( @transient var returnType: TypeInformation[CRow]) extends ProcessFunction[CRow, CRow] with ResultTypeQueryable[CRow] - with Compiler[Any] { - - val LOG: Logger = LoggerFactory.getLogger(this.getClass) + with Compiler[Any] + with Logging { private var function: ProcessFunction[Row, Row] = _ private var collector: TableFunctionCollector[_] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala index 9ed9188b8d4d8..54bac601bcc10 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala @@ -24,8 +24,8 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.Compiler import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row -import org.slf4j.LoggerFactory /** * MapRunner with [[CRow]] input. @@ -36,9 +36,8 @@ class CRowMapRunner[OUT]( @transient var returnType: TypeInformation[OUT]) extends RichMapFunction[CRow, OUT] with ResultTypeQueryable[OUT] - with Compiler[MapFunction[Row, OUT]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[MapFunction[Row, OUT]] + with Logging { private var function: MapFunction[Row, OUT] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala index c80f29102b82a..600b8987a28f4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala @@ -25,9 +25,9 @@ import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.streaming.api.operators.TimestampedCollector import org.apache.flink.table.codegen.Compiler import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * ProcessRunner with [[CRow]] output. @@ -38,9 +38,8 @@ class CRowOutputProcessRunner( @transient var returnType: TypeInformation[CRow]) extends ProcessFunction[Any, CRow] with ResultTypeQueryable[CRow] - with Compiler[ProcessFunction[Any, Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[ProcessFunction[Any, Row]] + with Logging { private var function: ProcessFunction[Any, Row] = _ private var cRowWrapper: CRowWrappingCollector = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowProcessRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowProcessRunner.scala index cef62a517be2f..a7f3d7287baff 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowProcessRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowProcessRunner.scala @@ -25,9 +25,9 @@ import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.codegen.Compiler import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * ProcessRunner with [[CRow]] input and [[CRow]] output. @@ -38,9 +38,8 @@ class CRowProcessRunner( @transient var returnType: TypeInformation[CRow]) extends ProcessFunction[CRow, CRow] with ResultTypeQueryable[CRow] - with Compiler[ProcessFunction[Row, Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[ProcessFunction[Row, Row]] + with Logging { private var function: ProcessFunction[Row, Row] = _ private var cRowWrapper: CRowWrappingCollector = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala index 478b6b64cfcad..e2f5e6113361f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CorrelateFlatMapRunner.scala @@ -24,8 +24,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.util.Logging import org.apache.flink.util.Collector -import org.slf4j.{Logger, LoggerFactory} class CorrelateFlatMapRunner[IN, OUT]( flatMapName: String, @@ -35,9 +35,8 @@ class CorrelateFlatMapRunner[IN, OUT]( @transient var returnType: TypeInformation[OUT]) extends RichFlatMapFunction[IN, OUT] with ResultTypeQueryable[OUT] - with Compiler[Any] { - - val LOG: Logger = LoggerFactory.getLogger(this.getClass) + with Compiler[Any] + with Logging { private var function: FlatMapFunction[IN, OUT] = _ private var collector: TableFunctionCollector[_] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala index 67acc0b10ee1d..0bf65694367f9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala @@ -21,10 +21,10 @@ package org.apache.flink.table.runtime import org.apache.flink.api.common.functions.{FlatJoinFunction, RichFlatJoinFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable -import org.apache.flink.table.codegen.Compiler import org.apache.flink.configuration.Configuration +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.util.Logging import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory class FlatJoinRunner[IN1, IN2, OUT]( name: String, @@ -32,9 +32,8 @@ class FlatJoinRunner[IN1, IN2, OUT]( @transient var returnType: TypeInformation[OUT]) extends RichFlatJoinFunction[IN1, IN2, OUT] with ResultTypeQueryable[OUT] - with Compiler[FlatJoinFunction[IN1, IN2, OUT]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[FlatJoinFunction[IN1, IN2, OUT]] + with Logging { private var function: FlatJoinFunction[IN1, IN2, OUT] = null diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala index 938da59ea4aa4..6c1f80489851e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala @@ -22,11 +22,11 @@ import org.apache.flink.api.common.functions.util.FunctionUtils import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable -import org.apache.flink.table.codegen.Compiler import org.apache.flink.configuration.Configuration +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory class FlatMapRunner( name: String, @@ -34,9 +34,8 @@ class FlatMapRunner( @transient var returnType: TypeInformation[Row]) extends RichFlatMapFunction[Row, Row] with ResultTypeQueryable[Row] - with Compiler[FlatMapFunction[Row, Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[FlatMapFunction[Row, Row]] + with Logging { private var function: FlatMapFunction[Row, Row] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala index 14eeecfb451b5..00d18ecc00794 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala @@ -21,9 +21,9 @@ package org.apache.flink.table.runtime import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable -import org.apache.flink.table.codegen.Compiler import org.apache.flink.configuration.Configuration -import org.slf4j.LoggerFactory +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.util.Logging class MapRunner[IN, OUT]( name: String, @@ -31,9 +31,8 @@ class MapRunner[IN, OUT]( @transient var returnType: TypeInformation[OUT]) extends RichMapFunction[IN, OUT] with ResultTypeQueryable[OUT] - with Compiler[MapFunction[IN, OUT]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[MapFunction[IN, OUT]] + with Logging { private var function: MapFunction[IN, OUT] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala index 00b7b8eee0ce1..5f5a2cc4a19a3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala @@ -21,9 +21,9 @@ package org.apache.flink.table.runtime import org.apache.flink.api.common.functions.{FlatJoinFunction, RichFlatMapFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable -import org.apache.flink.table.codegen.Compiler import org.apache.flink.configuration.Configuration -import org.slf4j.LoggerFactory +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.util.Logging abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, MULTI_IN, OUT]( name: String, @@ -32,9 +32,8 @@ abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, MULTI_IN, OUT]( broadcastSetName: String) extends RichFlatMapFunction[MULTI_IN, OUT] with ResultTypeQueryable[OUT] - with Compiler[FlatJoinFunction[IN1, IN2, OUT]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[FlatJoinFunction[IN1, IN2, OUT]] + with Logging { protected var function: FlatJoinFunction[IN1, IN2, OUT] = _ protected var broadcastSet: Option[SINGLE_IN] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala index dd9c015c2d98e..d3bffda284df9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAggFunction.scala @@ -21,8 +21,8 @@ package org.apache.flink.table.runtime.aggregate import org.apache.flink.api.common.functions.AggregateFunction import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row -import org.slf4j.LoggerFactory /** * Aggregate Function used for the aggregate operator in @@ -31,9 +31,8 @@ import org.slf4j.LoggerFactory * @param genAggregations Generated aggregate helper function */ class AggregateAggFunction(genAggregations: GeneratedAggregationsFunction) - extends AggregateFunction[CRow, Row, Row] with Compiler[GeneratedAggregations] { + extends AggregateFunction[CRow, Row, Row] with Compiler[GeneratedAggregations] with Logging { - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def createAccumulator(): Row = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala index 5f459f98a4aa2..83e1b1315ad05 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetAggFunction.scala @@ -22,9 +22,9 @@ import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * [[RichGroupReduceFunction]] to compute aggregates that do not support pre-aggregation for batch @@ -35,12 +35,11 @@ import org.slf4j.LoggerFactory class DataSetAggFunction( private val genAggregations: GeneratedAggregationsFunction) extends RichGroupReduceFunction[Row, Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] with Logging { private var output: Row = _ private var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetFinalAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetFinalAggFunction.scala index 9b81992cba24a..52762712b948c 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetFinalAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetFinalAggFunction.scala @@ -23,9 +23,9 @@ import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * [[RichGroupReduceFunction]] to compute the final result of a pre-aggregated aggregation @@ -36,12 +36,11 @@ import org.slf4j.LoggerFactory class DataSetFinalAggFunction( private val genAggregations: GeneratedAggregationsFunction) extends RichGroupReduceFunction[Row, Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] with Logging { private var output: Row = _ private var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetPreAggFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetPreAggFunction.scala index 8febe3e820ff4..fc3366bd31691 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetPreAggFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetPreAggFunction.scala @@ -22,9 +22,9 @@ import java.lang.Iterable import org.apache.flink.api.common.functions._ import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * [[GroupCombineFunction]] and [[MapPartitionFunction]] to compute pre-aggregates for batch @@ -36,12 +36,12 @@ class DataSetPreAggFunction(genAggregations: GeneratedAggregationsFunction) extends AbstractRichFunction with GroupCombineFunction[Row, Row] with MapPartitionFunction[Row, Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var output: Row = _ private var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala index 2160ef5bd5941..d99ca31df9626 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggReduceGroupFunction.scala @@ -20,11 +20,11 @@ package org.apache.flink.table.runtime.aggregate import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction -import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging +import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * It wraps the aggregate logic inside of @@ -54,7 +54,8 @@ class DataSetSessionWindowAggReduceGroupFunction( gap: Long, isInputCombined: Boolean) extends RichGroupReduceFunction[Row, Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var collector: RowTimeWindowPropertyCollector = _ private val intermediateRowWindowStartPos = keysAndAggregatesArity @@ -63,7 +64,6 @@ class DataSetSessionWindowAggReduceGroupFunction( private var output: Row = _ private var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala index 9bcac3031d52c..666bfee19348e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSessionWindowAggregatePreProcessor.scala @@ -22,11 +22,11 @@ import java.lang.Iterable import org.apache.flink.api.common.functions.{AbstractRichFunction, GroupCombineFunction, MapPartitionFunction} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable -import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging +import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * This wraps the aggregate logic inside of @@ -46,13 +46,13 @@ class DataSetSessionWindowAggregatePreProcessor( with MapPartitionFunction[Row,Row] with GroupCombineFunction[Row,Row] with ResultTypeQueryable[Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var output: Row = _ private val rowTimeFieldPos = keysAndAggregatesArity private var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggReduceGroupFunction.scala index b3a19a443328a..3af7969a37c06 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideTimeWindowAggReduceGroupFunction.scala @@ -25,9 +25,9 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.windowing.windows.TimeWindow import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * It is used for sliding windows on batch for time-windows. It takes a prepared input row (with @@ -53,7 +53,8 @@ class DataSetSlideTimeWindowAggReduceGroupFunction( extends RichGroupReduceFunction[Row, Row] with CombineFunction[Row, Row] with ResultTypeQueryable[Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private val timeFieldPos = returnType.getArity - 1 private val intermediateWindowStartPos = keysAndAggregatesArity @@ -61,7 +62,6 @@ class DataSetSlideTimeWindowAggReduceGroupFunction( protected var intermediateRow: Row = _ private var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala index e4b9458ed834e..c64a52217f130 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetSlideWindowAggReduceGroupFunction.scala @@ -22,9 +22,9 @@ import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * It wraps the aggregate logic inside of @@ -45,7 +45,8 @@ class DataSetSlideWindowAggReduceGroupFunction( finalRowWindowEndPos: Option[Int], windowSize: Long) extends RichGroupReduceFunction[Row, Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var collector: RowTimeWindowPropertyCollector = _ protected val windowStartPos: Int = keysAndAggregatesArity @@ -53,7 +54,6 @@ class DataSetSlideWindowAggReduceGroupFunction( private var output: Row = _ protected var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) protected var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala index 0e73f7b34baf2..22fe389a5b250 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleCountWindowAggReduceGroupFunction.scala @@ -22,9 +22,9 @@ import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * It wraps the aggregate logic inside of @@ -38,12 +38,12 @@ class DataSetTumbleCountWindowAggReduceGroupFunction( private val genAggregations: GeneratedAggregationsFunction, private val windowSize: Long) extends RichGroupReduceFunction[Row, Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var output: Row = _ private var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala index b4f7585282a14..7ae4c173e78cd 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetTumbleTimeWindowAggReduceGroupFunction.scala @@ -22,9 +22,9 @@ import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * It wraps the aggregate logic inside of @@ -44,7 +44,8 @@ class DataSetTumbleTimeWindowAggReduceGroupFunction( windowEndPos: Option[Int], keysAndAggregatesArity: Int) extends RichGroupReduceFunction[Row, Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var collector: RowTimeWindowPropertyCollector = _ protected var aggregateBuffer: Row = new Row(keysAndAggregatesArity + 1) @@ -52,7 +53,6 @@ class DataSetTumbleTimeWindowAggReduceGroupFunction( private var output: Row = _ protected var accumulators: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) protected var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetWindowAggMapFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetWindowAggMapFunction.scala index d49ed0e13ee39..324784fc45386 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetWindowAggMapFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/DataSetWindowAggMapFunction.scala @@ -26,8 +26,8 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.windowing.windows.TimeWindow import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row -import org.slf4j.LoggerFactory /** * This map function only works for windows on batch tables. @@ -44,12 +44,12 @@ class DataSetWindowAggMapFunction( @transient private val returnType: TypeInformation[Row]) extends RichMapFunction[Row, Row] with ResultTypeQueryable[Row] - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var accs: Row = _ private var output: Row = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala index a4769875997f9..df594608ea086 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/GroupAggProcessFunction.scala @@ -19,16 +19,16 @@ package org.apache.flink.table.runtime.aggregate import java.lang.{Long => JLong} +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor} +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.types.Row -import org.apache.flink.util.Collector -import org.apache.flink.api.common.state.{StateDescriptor, ValueState, ValueStateDescriptor} -import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.{StreamQueryConfig, Types} import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} -import org.slf4j.{Logger, LoggerFactory} import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging +import org.apache.flink.types.Row +import org.apache.flink.util.Collector /** * Aggregate Function used for the groupby (without window) aggregate @@ -42,9 +42,9 @@ class GroupAggProcessFunction( private val generateRetraction: Boolean, private val queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { - val LOG: Logger = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ private var newRow: CRow = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala index 5c2851904f5b6..1d947a029537a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRangeOver.scala @@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.slf4j.LoggerFactory +import org.apache.flink.table.util.Logging /** * Process Function used for the aggregate in bounded proc-time OVER window @@ -50,13 +50,13 @@ class ProcTimeBoundedRangeOver( inputType: TypeInformation[CRow], queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var output: CRow = _ private var accumulatorState: ValueState[Row] = _ private var rowMapState: MapState[Long, JList[Row]] = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala index 09d98adac8fff..ccddaa5b10fc7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeBoundedRowsOver.scala @@ -18,22 +18,19 @@ package org.apache.flink.table.runtime.aggregate import java.util +import java.util.{List => JList} +import org.apache.flink.api.common.state.{MapState, MapStateDescriptor, ValueState, ValueStateDescriptor} +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} +import org.apache.flink.api.java.typeutils.{ListTypeInfo, RowTypeInfo} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.types.Row -import org.apache.flink.util.{Collector, Preconditions} -import org.apache.flink.api.common.state._ -import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.java.typeutils.ListTypeInfo -import java.util.{List => JList} - -import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.slf4j.LoggerFactory +import org.apache.flink.table.util.Logging +import org.apache.flink.types.Row +import org.apache.flink.util.{Collector, Preconditions} /** * Process Function for ROW clause processing-time bounded OVER window @@ -50,7 +47,8 @@ class ProcTimeBoundedRowsOver( inputType: TypeInformation[CRow], queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { Preconditions.checkArgument(precedingOffset > 0) @@ -60,7 +58,6 @@ class ProcTimeBoundedRowsOver( private var counterState: ValueState[Long] = _ private var smallestTsState: ValueState[Long] = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala index 4fb559544d5db..6e4c510578682 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/ProcTimeUnboundedOver.scala @@ -17,16 +17,16 @@ */ package org.apache.flink.table.runtime.aggregate +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor} +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.types.Row -import org.apache.flink.util.Collector -import org.apache.flink.api.common.state.{StateDescriptor, ValueState, ValueStateDescriptor} -import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.CRow -import org.slf4j.LoggerFactory +import org.apache.flink.table.util.Logging +import org.apache.flink.types.Row +import org.apache.flink.util.Collector /** * Process Function for processing-time unbounded OVER window @@ -39,11 +39,11 @@ class ProcTimeUnboundedOver( aggregationStateType: RowTypeInfo, queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { private var output: CRow = _ private var state: ValueState[Row] = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala index 1ee269326ef55..85c523ea4b4fc 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRangeOver.scala @@ -28,9 +28,9 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.{Collector, Preconditions} -import org.slf4j.LoggerFactory /** * Process Function for RANGE clause event-time bounded OVER window @@ -48,7 +48,8 @@ class RowTimeBoundedRangeOver( rowTimeIdx: Int, queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { Preconditions.checkNotNull(aggregationStateType) Preconditions.checkNotNull(precedingOffset) @@ -66,7 +67,6 @@ class RowTimeBoundedRangeOver( // to this time stamp. private var dataState: MapState[Long, JList[Row]] = _ - val LOG = LoggerFactory.getLogger(this.getClass) private var function: GeneratedAggregations = _ override def open(config: Configuration) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala index 60200bcc1df9a..e120d6b0afda4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeBoundedRowsOver.scala @@ -29,9 +29,9 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.{Collector, Preconditions} -import org.slf4j.{Logger, LoggerFactory} /** * Process Function for ROWS clause event-time bounded OVER window @@ -49,13 +49,12 @@ class RowTimeBoundedRowsOver( rowTimeIdx: Int, queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) - with Compiler[GeneratedAggregations] { + with Compiler[GeneratedAggregations] + with Logging { Preconditions.checkNotNull(aggregationStateType) Preconditions.checkNotNull(precedingOffset) - val LOG: Logger = LoggerFactory.getLogger(this.getClass) - private var output: CRow = _ // the state which keeps the last triggering timestamp diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala index c8183c944b851..c8236a3d605f6 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala @@ -29,9 +29,9 @@ import org.apache.flink.streaming.api.operators.TimestampedCollector import org.apache.flink.table.api.StreamQueryConfig import org.apache.flink.table.codegen.{Compiler, GeneratedAggregationsFunction} import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.{Logger, LoggerFactory} /** @@ -48,9 +48,8 @@ abstract class RowTimeUnboundedOver( rowTimeIdx: Int, queryConfig: StreamQueryConfig) extends ProcessFunctionWithCleanupState[CRow, CRow](queryConfig) - with Compiler[GeneratedAggregations] { - - val LOG: Logger = LoggerFactory.getLogger(this.getClass) + with Compiler[GeneratedAggregations] + with Logging { protected var output: CRow = _ // state to hold the accumulators of the aggregations diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala index 95f304d734b42..a9966f8a562f4 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToJavaTupleMapRunner.scala @@ -27,8 +27,8 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.Compiler import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row -import org.slf4j.{Logger, LoggerFactory} /** * Convert [[CRow]] to a [[JTuple2]]. @@ -39,9 +39,8 @@ class CRowToJavaTupleMapRunner( @transient var returnType: TypeInformation[JTuple2[JBool, Any]]) extends RichMapFunction[CRow, Any] with ResultTypeQueryable[JTuple2[JBool, Any]] - with Compiler[MapFunction[Row, Any]] { - - val LOG: Logger = LoggerFactory.getLogger(this.getClass) + with Compiler[MapFunction[Row, Any]] + with Logging { private var function: MapFunction[Row, Any] = _ private var tupleWrapper: JTuple2[JBool, Any] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapRunner.scala index c7d71a9e46e68..5b122f3176e1d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapRunner.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/conversion/CRowToScalaTupleMapRunner.scala @@ -24,8 +24,8 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.configuration.Configuration import org.apache.flink.table.codegen.Compiler import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row -import org.slf4j.{Logger, LoggerFactory} /** * Convert [[CRow]] to a [[Tuple2]]. @@ -36,9 +36,8 @@ class CRowToScalaTupleMapRunner( @transient var returnType: TypeInformation[(Boolean, Any)]) extends RichMapFunction[CRow, (Boolean, Any)] with ResultTypeQueryable[(Boolean, Any)] - with Compiler[MapFunction[Row, Any]] { - - val LOG: Logger = LoggerFactory.getLogger(this.getClass) + with Compiler[MapFunction[Row, Any]] + with Logging { private var function: MapFunction[Row, Any] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala index 1cb3a6e0283ce..fff5924931871 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/CRowValuesInputFormat.scala @@ -24,8 +24,8 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable import org.apache.flink.core.io.GenericInputSplit import org.apache.flink.table.codegen.Compiler import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row -import org.slf4j.LoggerFactory class CRowValuesInputFormat( name: String, @@ -34,9 +34,8 @@ class CRowValuesInputFormat( extends GenericInputFormat[CRow] with NonParallelInput with ResultTypeQueryable[CRow] - with Compiler[GenericInputFormat[Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[GenericInputFormat[Row]] + with Logging { private var format: GenericInputFormat[Row] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala index 43ce6056610cd..858146a450a83 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala @@ -21,10 +21,10 @@ package org.apache.flink.table.runtime.io import org.apache.flink.api.common.io.{GenericInputFormat, NonParallelInput} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable -import org.apache.flink.table.codegen.Compiler import org.apache.flink.core.io.GenericInputSplit +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row -import org.slf4j.LoggerFactory class ValuesInputFormat( name: String, @@ -33,9 +33,8 @@ class ValuesInputFormat( extends GenericInputFormat[Row] with NonParallelInput with ResultTypeQueryable[Row] - with Compiler[GenericInputFormat[Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) + with Compiler[GenericInputFormat[Row]] + with Logging { private var format: GenericInputFormat[Row] = _ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala index e62a18f05816d..824037630f285 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala @@ -30,9 +30,9 @@ import org.apache.flink.streaming.api.functions.co.CoProcessFunction import org.apache.flink.table.codegen.Compiler import org.apache.flink.table.runtime.CRowWrappingCollector import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging import org.apache.flink.types.Row import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory /** * A CoProcessFunction to support stream join stream, currently just support inner-join @@ -55,7 +55,8 @@ class ProcTimeWindowInnerJoin( private val genJoinFuncName: String, private val genJoinFuncCode: String) extends CoProcessFunction[CRow, CRow, CRow] - with Compiler[FlatJoinFunction[Row, Row, Row]]{ + with Compiler[FlatJoinFunction[Row, Row, Row]] + with Logging { private var cRowWrapper: CRowWrappingCollector = _ @@ -80,8 +81,6 @@ class ProcTimeWindowInnerJoin( private val leftStreamWinSize: Long = if (leftLowerBound <= 0) -leftLowerBound else -1 private val rightStreamWinSize: Long = if (leftUpperBound >= 0) leftUpperBound else -1 - val LOG = LoggerFactory.getLogger(this.getClass) - override def open(config: Configuration) { LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " + s"Code:\n$genJoinFuncCode") diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/util/Logging.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/util/Logging.scala new file mode 100644 index 0000000000000..b6be99e6710d9 --- /dev/null +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/util/Logging.scala @@ -0,0 +1,28 @@ +/* + * 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.util + +import org.slf4j.{Logger, LoggerFactory} + +/** + * Helper class to ensure the logger is never serialized. + */ +trait Logging { + @transient lazy val LOG: Logger = LoggerFactory.getLogger(getClass) +} From b915757f5a2417b51eeb44f6dad0a9aae9050a72 Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Thu, 10 Aug 2017 15:09:56 -0700 Subject: [PATCH 068/129] FLINK-7366 Upgrade kinesis-producer-library in flink-connector-kinesis from 0.10.2 to 0.12.5 This closes #4522 --- 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 41daaa7f6ca95..c5a91b178c2a0 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -35,7 +35,7 @@ under the License. 1.10.71 1.6.2 - 0.10.2 + 0.12.5 jar From 1b7f8bd82d500b310fc90430a0ca419625e92e62 Mon Sep 17 00:00:00 2001 From: yew1eb Date: Wed, 30 Aug 2017 18:04:26 +0800 Subject: [PATCH 069/129] [FLINK-7559] [quickstart] fix Typo in flink-quickstart pom This closes #4619. --- .../src/main/resources/archetype-resources/pom.xml | 2 +- .../src/main/resources/archetype-resources/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml index 5da38cdef9503..c973a6a3406ce 100644 --- a/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-java/src/main/resources/archetype-resources/pom.xml @@ -92,7 +92,7 @@ under the License. ${flink.version} - org.slf4j diff --git a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml index 67fe4c1d91da3..42d7cdb3a496a 100644 --- a/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/src/main/resources/archetype-resources/pom.xml @@ -93,7 +93,7 @@ under the License. ${flink.version} - org.slf4j From e753db8411debfc573ffc330355a0f24c0afbfb5 Mon Sep 17 00:00:00 2001 From: Nycholas de Oliveira e Oliveira Date: Tue, 29 Aug 2017 14:21:03 -0300 Subject: [PATCH 070/129] [FLINK-7556] Allow Integer.MIN_VALUE for fetch size in JDBCInputFormat Allow Integer.MIN_VALUE to be accepted as a parameter for setFetchSize for MySQL Driver. The combination of a forward-only, read-only result set, with a fetch size of Integer.MIN_VALUE serves as a signal to the driver to stream result sets row-by-row. After this, any result sets created with the statement will be retrieved row-by-row. This closes #4617. --- .../flink/api/java/io/jdbc/JDBCInputFormat.java | 5 +++-- .../flink/api/java/io/jdbc/JDBCInputFormatTest.java | 11 +++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java index b7ac7446a410d..7d088147ae829 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java @@ -144,7 +144,7 @@ public void openInputFormat() { dbConn = DriverManager.getConnection(dbURL, username, password); } statement = dbConn.prepareStatement(queryTemplate, resultSetType, resultSetConcurrency); - if (fetchSize > 0) { + if (fetchSize == Integer.MIN_VALUE || fetchSize > 0) { statement.setFetchSize(fetchSize); } } catch (SQLException se) { @@ -390,7 +390,8 @@ public JDBCInputFormatBuilder setRowTypeInfo(RowTypeInfo rowTypeInfo) { } public JDBCInputFormatBuilder setFetchSize(int fetchSize) { - Preconditions.checkArgument(fetchSize > 0, "Illegal value %s for fetchSize, has to be positive.", fetchSize); + Preconditions.checkArgument(fetchSize == Integer.MIN_VALUE || fetchSize > 0, + "Illegal value %s for fetchSize, has to be positive or Integer.MIN_VALUE.", fetchSize); format.fetchSize = fetchSize; return this; } diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java index f7a86e5afd222..10e8c66a7ddfa 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java @@ -113,6 +113,17 @@ public void testInvalidFetchSize() { .finish(); } + @Test + public void testValidFetchSizeIntegerMin() { + jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() + .setDrivername(DRIVER_CLASS) + .setDBUrl(DB_URL) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .setFetchSize(Integer.MIN_VALUE) + .finish(); + } + @Test public void testDefaultFetchSizeIsUsedIfNotConfiguredOtherwise() throws SQLException, ClassNotFoundException { jdbcInputFormat = JDBCInputFormat.buildJDBCInputFormat() From e6fddbc3e5bdb42764191990ac29ba382bbe6016 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=87=91=E7=AB=B9?= Date: Wed, 30 Aug 2017 16:55:09 +0800 Subject: [PATCH 071/129] [FLINK-7558][table]Improve SQL ValidationException message. This closes #4620 --- .../functions/utils/AggSqlFunction.scala | 4 +- .../functions/utils/ScalarSqlFunction.scala | 9 ++- .../UserDefinedFunctionValidationTest.scala | 61 +++++++++++++++++++ 3 files changed, 71 insertions(+), 3 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala index 526ec47d03e4f..bb71d6362e9cf 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala @@ -95,7 +95,9 @@ object AggSqlFunction { val foundSignature = getAccumulateMethodSignature(aggregateFunction, operandTypeInfo) .getOrElse( throw new ValidationException( - s"Operand types of ${signatureToString(operandTypeInfo)} could not be inferred.")) + s"Given parameters of function do not match any signature. \n" + + s"Actual: ${signatureToString(operandTypeInfo)} \n" + + s"Expected: ${signaturesToString(aggregateFunction, "accumulate")}")) val inferredTypes = getParameterTypes(aggregateFunction, foundSignature.drop(1)) .map(typeFactory.createTypeFromTypeInfo(_, isNullable = true)) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala index 0776f7af26446..784bca74de77f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala @@ -45,7 +45,7 @@ class ScalarSqlFunction( extends SqlFunction( new SqlIdentifier(name, SqlParserPos.ZERO), createReturnTypeInference(name, scalarFunction, typeFactory), - createOperandTypeInference(scalarFunction, typeFactory), + createOperandTypeInference(name, scalarFunction, typeFactory), createOperandTypeChecker(name, scalarFunction), null, SqlFunctionCategory.USER_DEFINED_FUNCTION) { @@ -91,6 +91,7 @@ object ScalarSqlFunction { } private[flink] def createOperandTypeInference( + name: String, scalarFunction: ScalarFunction, typeFactory: FlinkTypeFactory) : SqlOperandTypeInference = { @@ -106,7 +107,11 @@ object ScalarSqlFunction { val operandTypeInfo = getOperandTypeInfo(callBinding) val foundSignature = getEvalMethodSignature(scalarFunction, operandTypeInfo) - .getOrElse(throw new ValidationException(s"Operand types of could not be inferred.")) + .getOrElse( + throw new ValidationException( + s"Given parameters of function '$name' do not match any signature. \n" + + s"Actual: ${signatureToString(operandTypeInfo)} \n" + + s"Expected: ${signaturesToString(scalarFunction, "eval")}")) val inferredTypes = scalarFunction .getParameterTypes(foundSignature) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala new file mode 100644 index 0000000000000..6939771b1a0d3 --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala @@ -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.table.api.validation + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.ValidationException +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.expressions.utils.Func0 +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvg +import org.apache.flink.table.utils.TableTestBase +import org.junit.Test + +class UserDefinedFunctionValidationTest extends TableTestBase { + + @Test + def testScalarFunctionOperandTypeCheck(): Unit = { + thrown.expect(classOf[ValidationException]) + thrown.expectMessage( + "Given parameters of function 'func' do not match any signature. \n" + + "Actual: (java.lang.String) \n" + + "Expected: (int)") + val util = streamTestUtil() + util.addTable[(Int, String)]("t", 'a, 'b) + util.tableEnv.registerFunction("func",Func0) + util.verifySql("select func(b) from t","n/a") + } + + @Test + def testAggregateFunctionOperandTypeCheck(): Unit = { + thrown.expect(classOf[ValidationException]) + thrown.expectMessage( + "Given parameters of function do not match any signature. \n" + + "Actual: (java.lang.String, java.lang.Integer) \n" + + "Expected: (org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions." + + "WeightedAvgAccum, int, int), (org.apache.flink.table.runtime.utils." + + "JavaUserDefinedAggFunctions.WeightedAvgAccum, long, int)") + + val util = streamTestUtil() + val weightAvgFun = new WeightedAvg + util.addTable[(Int, String)]("t", 'a, 'b) + util.tableEnv.registerFunction("agg",weightAvgFun) + util.verifySql("select agg(b, a) from t","n/a") + } + +} + From ddf62b114d24284208774ef7a52c89728434afbb Mon Sep 17 00:00:00 2001 From: zjureel Date: Wed, 9 Aug 2017 13:49:02 +0800 Subject: [PATCH 072/129] [FLINK-6787] Fix Job-/StoppableException extend FlinkException --- .../org/apache/flink/client/program/ClusterClient.java | 2 +- .../flink/client/program/ProgramMissingJobException.java | 8 +++++++- .../main/java/org/apache/flink/runtime/JobException.java | 4 +++- .../java/org/apache/flink/runtime/StoppingException.java | 4 +++- .../client/JobClientActorConnectionTimeoutException.java | 4 +++- .../JobClientActorRegistrationTimeoutException.java | 4 +++- .../client/JobClientActorSubmissionTimeoutException.java | 4 +++- .../flink/runtime/client/JobExecutionException.java | 3 ++- .../flink/runtime/jobmaster/JobManagerException.java | 4 +++- 9 files changed, 28 insertions(+), 9 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 7bc26550037f2..c8a236e52dea6 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 @@ -382,7 +382,7 @@ else if (prog.isUsingInteractiveMode()) { // invoke main method prog.invokeInteractiveModeForExecution(); if (lastJobExecutionResult == null && factory.getLastEnvCreated() == null) { - throw new ProgramMissingJobException(); + throw new ProgramMissingJobException("The program didn't contain a Flink job."); } if (isDetached()) { // in detached mode, we execute the whole user code to extract the Flink job, afterwards we run it here diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ProgramMissingJobException.java b/flink-clients/src/main/java/org/apache/flink/client/program/ProgramMissingJobException.java index 43d608b43aa2f..c2b57178eafbd 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/ProgramMissingJobException.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/ProgramMissingJobException.java @@ -18,12 +18,18 @@ package org.apache.flink.client.program; +import org.apache.flink.util.FlinkException; + /** * Exception used to indicate that no job was executed during the invocation of a Flink program. */ -public class ProgramMissingJobException extends Exception { +public class ProgramMissingJobException extends FlinkException { /** * Serial version UID for serialization interoperability. */ private static final long serialVersionUID = -1964276369605091101L; + + public ProgramMissingJobException(String message) { + super(message); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/JobException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/JobException.java index d5a5bb968acb4..d923af96f77e4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/JobException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/JobException.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime; +import org.apache.flink.util.FlinkException; + /** * Indicates that a job has failed. */ -public class JobException extends Exception { +public class JobException extends FlinkException { private static final long serialVersionUID = 1275864691743020176L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/StoppingException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/StoppingException.java index 6bb71ce6cfac5..3644219abfcef 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/StoppingException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/StoppingException.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime; +import org.apache.flink.util.FlinkException; + /** * Indicates that a job is not stoppable. */ -public class StoppingException extends Exception { +public class StoppingException extends FlinkException { private static final long serialVersionUID = -721315728140810694L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java index 72a56585f98ca..74a4e1f9cf783 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorConnectionTimeoutException.java @@ -18,11 +18,13 @@ package org.apache.flink.runtime.client; +import org.apache.flink.util.FlinkException; + /** * Exception which is thrown when the {@link JobClientActor} wants to submit a job to * the job manager but has not found one after a given timeout interval. */ -public class JobClientActorConnectionTimeoutException extends Exception { +public class JobClientActorConnectionTimeoutException extends FlinkException { private static final long serialVersionUID = 2287747430528388637L; public JobClientActorConnectionTimeoutException(String msg) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorRegistrationTimeoutException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorRegistrationTimeoutException.java index e57d1b4f40e19..499c9e40902a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorRegistrationTimeoutException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorRegistrationTimeoutException.java @@ -18,11 +18,13 @@ package org.apache.flink.runtime.client; +import org.apache.flink.util.FlinkException; + /** * Exception which is thrown by the {@link JobClientActor} if it has not heard back from the job * manager after it has attempted to register for a job within a given timeout interval. */ -public class JobClientActorRegistrationTimeoutException extends Exception { +public class JobClientActorRegistrationTimeoutException extends FlinkException { private static final long serialVersionUID = 8762463142030454853L; public JobClientActorRegistrationTimeoutException(String msg) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java index 2d394621abd53..a56e38990164c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClientActorSubmissionTimeoutException.java @@ -18,11 +18,13 @@ package org.apache.flink.runtime.client; +import org.apache.flink.util.FlinkException; + /** * Exception which is thrown by the {@link JobClientActor} if it has not heard back from the job * manager after it has submitted a job to it within a given timeout interval. */ -public class JobClientActorSubmissionTimeoutException extends Exception { +public class JobClientActorSubmissionTimeoutException extends FlinkException { private static final long serialVersionUID = 8762463142030454853L; public JobClientActorSubmissionTimeoutException(String msg) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java index 7c6a4afe64eff..47eaaf53eda60 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobExecutionException.java @@ -19,13 +19,14 @@ package org.apache.flink.runtime.client; import org.apache.flink.api.common.JobID; +import org.apache.flink.util.FlinkException; /** * This exception is the base exception for all exceptions that denote any failure during * the execution of a job. The JobExecutionException and its subclasses are thrown by * the {@link JobClient}. */ -public class JobExecutionException extends Exception { +public class JobExecutionException extends FlinkException { private static final long serialVersionUID = 2818087325120827525L; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java index bc2759d0ed0c1..1650c83290d3c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobManagerException.java @@ -18,10 +18,12 @@ package org.apache.flink.runtime.jobmaster; +import org.apache.flink.util.FlinkException; + /** * Base exception thrown by the {@link JobMaster}. */ -public class JobManagerException extends Exception { +public class JobManagerException extends FlinkException { private static final long serialVersionUID = -7290962952242188064L; From d8d061b8857f0f6c66e8efb4e1ff2ca33ada2008 Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Thu, 31 Aug 2017 20:25:44 +0800 Subject: [PATCH 073/129] [hotfix][table] Fix bug of testAggregateFunctionOperandTypeCheck This closes #4629 --- .../UserDefinedFunctionValidationTest.scala | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala index 6939771b1a0d3..aeb226eb35a80 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/UserDefinedFunctionValidationTest.scala @@ -21,7 +21,7 @@ import org.apache.flink.api.scala._ import org.apache.flink.table.api.ValidationException import org.apache.flink.table.api.scala._ import org.apache.flink.table.expressions.utils.Func0 -import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.WeightedAvg +import org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions.OverAgg0 import org.apache.flink.table.utils.TableTestBase import org.junit.Test @@ -36,8 +36,8 @@ class UserDefinedFunctionValidationTest extends TableTestBase { "Expected: (int)") val util = streamTestUtil() util.addTable[(Int, String)]("t", 'a, 'b) - util.tableEnv.registerFunction("func",Func0) - util.verifySql("select func(b) from t","n/a") + util.tableEnv.registerFunction("func", Func0) + util.verifySql("select func(b) from t", "n/a") } @Test @@ -46,15 +46,14 @@ class UserDefinedFunctionValidationTest extends TableTestBase { thrown.expectMessage( "Given parameters of function do not match any signature. \n" + "Actual: (java.lang.String, java.lang.Integer) \n" + - "Expected: (org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions." + - "WeightedAvgAccum, int, int), (org.apache.flink.table.runtime.utils." + - "JavaUserDefinedAggFunctions.WeightedAvgAccum, long, int)") + "Expected: (org.apache.flink.table.runtime.utils.JavaUserDefinedAggFunctions" + + ".Accumulator0, long, int)") val util = streamTestUtil() - val weightAvgFun = new WeightedAvg + val agg = new OverAgg0 util.addTable[(Int, String)]("t", 'a, 'b) - util.tableEnv.registerFunction("agg",weightAvgFun) - util.verifySql("select agg(b, a) from t","n/a") + util.tableEnv.registerFunction("agg", agg) + util.verifySql("select agg(b, a) from t", "n/a") } } From 391e39bc7647625c7ea59b5c3ce2cf23e891c550 Mon Sep 17 00:00:00 2001 From: Vishnu Viswanath Date: Mon, 31 Jul 2017 00:58:28 -0400 Subject: [PATCH 074/129] [FLINK-7299][AVRO] Write GenericRecord using AvroOutputFormat --- .../flink/api/java/io/AvroOutputFormat.java | 7 +++ .../api/java/io/AvroOutputFormatTest.java | 43 +++++++++++++++++++ 2 files changed, 50 insertions(+) diff --git a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java index aed40bf4d85f0..5da8f75dc9a8b 100644 --- a/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java +++ b/flink-connectors/flink-avro/src/main/java/org/apache/flink/api/java/io/AvroOutputFormat.java @@ -25,6 +25,7 @@ import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.io.DatumWriter; import org.apache.avro.reflect.ReflectData; import org.apache.avro.reflect.ReflectDatumWriter; @@ -134,6 +135,12 @@ public void open(int taskNumber, int numTasks) throws IOException { } catch (InstantiationException | IllegalAccessException e) { throw new RuntimeException(e.getMessage()); } + } else if (org.apache.avro.generic.GenericRecord.class.isAssignableFrom(avroValueType)) { + if (userDefinedSchema == null) { + throw new IllegalStateException("Schema must be set when using Generic Record"); + } + datumWriter = new GenericDatumWriter(userDefinedSchema); + schema = userDefinedSchema; } else { datumWriter = new ReflectDatumWriter(avroValueType); schema = ReflectData.get().getSchema(avroValueType); diff --git a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java index 87334a74ef97d..71ebd785bbde1 100644 --- a/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java +++ b/flink-connectors/flink-avro/src/test/java/org/apache/flink/api/java/io/AvroOutputFormatTest.java @@ -24,6 +24,10 @@ import org.apache.flink.core.fs.Path; import org.apache.avro.Schema; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; import org.junit.Test; import org.mockito.internal.util.reflection.Whitebox; @@ -35,6 +39,7 @@ import java.io.ObjectOutputStream; import static org.apache.flink.api.java.io.AvroOutputFormat.Codec; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -151,4 +156,42 @@ private void output(final AvroOutputFormat outputFormat) throws IOExceptio } outputFormat.close(); } + + @Test + public void testGenericRecord() throws IOException { + final Path outputPath = new Path(File.createTempFile("avro-output-file", "generic.avro").getAbsolutePath()); + final AvroOutputFormat outputFormat = new AvroOutputFormat<>(outputPath, GenericRecord.class); + Schema schema = new Schema.Parser().parse("{\"type\":\"record\", \"name\":\"user\", \"fields\": [{\"name\":\"user_name\", \"type\":\"string\"}, {\"name\":\"favorite_number\", \"type\":\"int\"}, {\"name\":\"favorite_color\", \"type\":\"string\"}]}"); + outputFormat.setWriteMode(FileSystem.WriteMode.OVERWRITE); + outputFormat.setSchema(schema); + output(outputFormat, schema); + + GenericDatumReader reader = new GenericDatumReader<>(schema); + DataFileReader dataFileReader = new DataFileReader<>(new File(outputPath.getPath()), reader); + + while (dataFileReader.hasNext()) { + GenericRecord record = dataFileReader.next(); + assertEquals(record.get("user_name").toString(), "testUser"); + assertEquals(record.get("favorite_number"), 1); + assertEquals(record.get("favorite_color").toString(), "blue"); + } + + //cleanup + FileSystem fs = FileSystem.getLocalFileSystem(); + fs.delete(outputPath, false); + + } + + private void output(final AvroOutputFormat outputFormat, Schema schema) throws IOException { + outputFormat.configure(new Configuration()); + outputFormat.open(1, 1); + for (int i = 0; i < 100; i++) { + GenericRecord record = new GenericData.Record(schema); + record.put("user_name", "testUser"); + record.put("favorite_number", 1); + record.put("favorite_color", "blue"); + outputFormat.writeRecord(record); + } + outputFormat.close(); + } } From c87275154cec0b305b3210a9244927d18b7ec26e Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Thu, 10 Aug 2017 16:31:50 -0700 Subject: [PATCH 075/129] [FLINK-7422] Upgrade Kinesis Client Library (KCL) and AWS SDK in flink-connector-kinesis --- flink-connectors/flink-connector-kinesis/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index c5a91b178c2a0..83934f64608be 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -33,8 +33,8 @@ under the License. flink-connector-kinesis_${scala.binary.version} flink-connector-kinesis - 1.10.71 - 1.6.2 + 1.11.171 + 1.8.1 0.12.5 From a00830318eb535743012fcc5baae23b51dc7f606 Mon Sep 17 00:00:00 2001 From: Yestin <873915231@qq.com> Date: Fri, 14 Jul 2017 16:41:51 +0800 Subject: [PATCH 076/129] [FLINK-7169][CEP] Support AFTER MATCH SKIP function in CEP This closes #4331 --- docs/dev/libs/cep.md | 99 ++++ .../flink/cep/scala/PatternStream.scala | 2 + .../flink/cep/scala/pattern/Pattern.scala | 32 ++ .../flink/cep/nfa/AfterMatchSkipStrategy.java | 139 +++++ .../java/org/apache/flink/cep/nfa/NFA.java | 93 +++- .../apache/flink/cep/nfa/SharedBuffer.java | 3 +- .../flink/cep/nfa/compiler/NFACompiler.java | 28 + .../AbstractKeyedCEPPatternOperator.java | 12 +- .../flink/cep/operator/CEPOperatorUtils.java | 62 +-- .../cep/operator/FlatSelectCepOperator.java | 6 +- .../FlatSelectTimeoutCepOperator.java | 7 +- .../flink/cep/operator/SelectCepOperator.java | 6 +- .../operator/SelectTimeoutCepOperator.java | 7 +- .../flink/cep/pattern/GroupPattern.java | 11 +- .../org/apache/flink/cep/pattern/Pattern.java | 67 ++- .../flink/cep/nfa/AfterMatchSkipITCase.java | 484 ++++++++++++++++++ .../org/apache/flink/cep/nfa/NFAITCase.java | 72 +++ .../flink/cep/nfa/NFATestUtilities.java | 8 +- .../flink/cep/nfa/SharedBufferTest.java | 46 ++ .../cep/nfa/compiler/NFACompilerTest.java | 32 ++ .../flink/cep/operator/CEPOperatorTest.java | 4 +- .../flink/cep/operator/CEPRescalingTest.java | 4 +- .../operator/CepOperatorTestUtilities.java | 34 +- 23 files changed, 1136 insertions(+), 122 deletions(-) create mode 100644 flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/AfterMatchSkipStrategy.java create mode 100644 flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java diff --git a/docs/dev/libs/cep.md b/docs/dev/libs/cep.md index 91125cad6b37c..492f95fb8bd0a 100644 --- a/docs/dev/libs/cep.md +++ b/docs/dev/libs/cep.md @@ -1250,6 +1250,105 @@ pattern.within(Time.seconds(10)) +### After Match Skip Strategy + +For a given pattern, same event may be assigned to multiple successful matches. In order to control to how many matches an event will be assigned, we need to specify the skip strategy called `AfterMatchSkipStrategy`. +There're four types of skip strategies, listed as follows: + +* *NO_SKIP*: Every possible match will be emitted. +* *SKIP_PAST_LAST_EVENT*: Discards every partial match that contains event of the match. +* *SKIP_TO_FIRST*: Discards every partial match that contains event of the match preceding the first of *PatternName*. +* *SKIP_TO_LAST*: Discards every partial match that contains event of the match preceding the last of *PatternName*. + +Notice that when using *SKIP_TO_FIRST* and *SKIP_TO_LAST* skip strategy, a valid *PatternName* should also be specified. + +Let's take an example: For a given pattern `a b{2}` and a data stream `ab1, ab2, ab3, ab4, ab5, ab6`, the differences between these four skip strategies can be listed as follows: + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Skip StrategyResult Description
NO_SKIP + ab1 ab2 ab3
+ ab2 ab3 ab4
+ ab3 ab4 ab5
+ ab4 ab5 ab6
+
After found matching ab1 ab2 ab3, the match process will not discard any result.
SKIP_PAST_LAST_EVENT + ab1 ab2 ab3
+ ab4 ab5 ab6
+
After found matching ab1 ab2 ab3, the match process will discard all started partial matches.
SKIP_TO_FIRST[b] + ab1 ab2 ab3
+ ab2 ab3 ab4
+ ab3 ab4 ab5
+ ab4 ab5 ab6
+
After found matching ab1 ab2 ab3, the match process will discard all partial matches containing ab1, which is the only event that comes before the first b.
SKIP_TO_LAST[b] + ab1 ab2 ab3
+ ab3 ab4 ab5
+
After found matching ab1 ab2 ab3, the match process will discard all partial matches containing ab1 and ab2, which are events that comes before the last b.
+ +To specify which skip strategy to use, just create an `AfterMatchSkipStrategy` by calling: + + + + + + + + + + + + + + + + + + + + + +
FunctionDescription
AfterMatchSkipStrategy.noSkip()Create a NO_SKIP skip strategy
AfterMatchSkipStrategy.skipPastLastEvent()Create a SKIP_PAST_LAST_EVENT skip strategy
AfterMatchSkipStrategy.skipToFirst(patternName)Create a SKIP_TO_FIRST skip strategy with the referenced pattern name patternName
AfterMatchSkipStrategy.skipToLast(patternName)Create a SKIP_TO_LAST skip strategy with the referenced pattern name patternName
+ +Then apply the skip strategy to a pattern by calling: + +
+
+{% highlight java %} +AfterMatchSkipStrategy skipStrategy = ... +Pattern.begin("patternName", skipStrategy); +{% endhighlight %} +
+
+{% highlight scala %} +val skipStrategy = ... +Pattern.begin("patternName", skipStrategy) +{% endhighlight %} +
+
+ ## Detecting Patterns After specifying the pattern sequence you are looking for, it is time to apply it to your input stream to detect diff --git a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/PatternStream.scala b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/PatternStream.scala index eacfa877d8ba9..d1b07b3519e76 100644 --- a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/PatternStream.scala +++ b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/PatternStream.scala @@ -26,6 +26,8 @@ import org.apache.flink.cep.{EventComparator, PatternFlatSelectFunction, Pattern import org.apache.flink.streaming.api.scala.{asScalaStream, _} import org.apache.flink.util.Collector +import org.apache.flink.cep.operator.CEPOperatorUtils +import org.apache.flink.cep.scala.pattern.Pattern import scala.collection.Map /** diff --git a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala index dba328cf92807..42a95e82c286e 100644 --- a/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala +++ b/flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/pattern/Pattern.scala @@ -18,6 +18,7 @@ package org.apache.flink.cep.scala.pattern import org.apache.flink.cep +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy import org.apache.flink.cep.pattern.conditions.IterativeCondition.{Context => JContext} import org.apache.flink.cep.pattern.conditions.{IterativeCondition, SimpleCondition} import org.apache.flink.cep.pattern.{MalformedPatternException, Quantifier, Pattern => JPattern} @@ -478,6 +479,12 @@ class Pattern[T , F <: T](jPattern: JPattern[T, F]) { def next(pattern: Pattern[T, F]): GroupPattern[T, F] = GroupPattern[T, F](jPattern.next(pattern.wrappedPattern)) + /** + * Get after match skip strategy. + * @return current after match skip strategy + */ + def getAfterMatchSkipStrategy: AfterMatchSkipStrategy = + jPattern.getAfterMatchSkipStrategy } object Pattern { @@ -502,6 +509,18 @@ object Pattern { */ def begin[X](name: String): Pattern[X, X] = Pattern(JPattern.begin(name)) + /** + * Starts a new pattern sequence. The provided name is the one of the initial pattern + * of the new sequence. Furthermore, the base type of the event sequence is set. + * + * @param name The name of starting pattern of the new pattern sequence + * @param afterMatchSkipStrategy The skip strategy to use after each match + * @tparam X Base type of the event pattern + * @return The first pattern of a pattern sequence + */ + def begin[X](name: String, afterMatchSkipStrategy: AfterMatchSkipStrategy): Pattern[X, X] = + Pattern(JPattern.begin(name, afterMatchSkipStrategy)) + /** * Starts a new pattern sequence. The provided pattern is the initial pattern * of the new sequence. @@ -511,4 +530,17 @@ object Pattern { */ def begin[T, F <: T](pattern: Pattern[T, F]): GroupPattern[T, F] = GroupPattern[T, F](JPattern.begin(pattern.wrappedPattern)) + + /** + * Starts a new pattern sequence. The provided pattern is the initial pattern + * of the new sequence. + * + * @param pattern the pattern to begin with + * @param afterMatchSkipStrategy The skip strategy to use after each match + * @return The first pattern of a pattern sequence + */ + def begin[T, F <: T](pattern: Pattern[T, F], + afterMatchSkipStrategy: AfterMatchSkipStrategy): GroupPattern[T, F] = + GroupPattern(JPattern.begin(pattern.wrappedPattern, afterMatchSkipStrategy)) + } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/AfterMatchSkipStrategy.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/AfterMatchSkipStrategy.java new file mode 100644 index 0000000000000..dcda441a45dc9 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/AfterMatchSkipStrategy.java @@ -0,0 +1,139 @@ +/* + * 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.cep.nfa; + +import java.io.Serializable; + + +/** + * Indicate the skip strategy after a match process. + * + *

For more info on possible skip strategies see {@link SkipStrategy}. + */ +public class AfterMatchSkipStrategy implements Serializable { + + private static final long serialVersionUID = -4048930333619068531L; + // default strategy + private SkipStrategy strategy = SkipStrategy.NO_SKIP; + + // pattern name to skip to + private String patternName = null; + + /** + * Discards every partial match that contains event of the match preceding the first of *PatternName*. + * @param patternName the pattern name to skip to + * @return the created AfterMatchSkipStrategy + */ + public static AfterMatchSkipStrategy skipToFirst(String patternName) { + return new AfterMatchSkipStrategy(SkipStrategy.SKIP_TO_FIRST, patternName); + } + + /** + * Discards every partial match that contains event of the match preceding the last of *PatternName*. + * @param patternName the pattern name to skip to + * @return the created AfterMatchSkipStrategy + */ + public static AfterMatchSkipStrategy skipToLast(String patternName) { + return new AfterMatchSkipStrategy(SkipStrategy.SKIP_TO_LAST, patternName); + } + + /** + * Discards every partial match that contains event of the match. + * @return the created AfterMatchSkipStrategy + */ + public static AfterMatchSkipStrategy skipPastLastEvent() { + return new AfterMatchSkipStrategy(SkipStrategy.SKIP_PAST_LAST_EVENT); + } + + /** + * Every possible match will be emitted. + * @return the created AfterMatchSkipStrategy + */ + public static AfterMatchSkipStrategy noSkip() { + return new AfterMatchSkipStrategy(SkipStrategy.NO_SKIP); + } + + private AfterMatchSkipStrategy(SkipStrategy strategy) { + this(strategy, null); + } + + private AfterMatchSkipStrategy(SkipStrategy strategy, String patternName) { + if (patternName == null && (strategy == SkipStrategy.SKIP_TO_FIRST || strategy == SkipStrategy.SKIP_TO_LAST)) { + throw new IllegalArgumentException("The patternName field can not be empty when SkipStrategy is " + strategy); + } + this.strategy = strategy; + this.patternName = patternName; + } + + /** + * Get the {@link SkipStrategy} enum. + * @return the skip strategy + */ + public SkipStrategy getStrategy() { + return strategy; + } + + /** + * Get the referenced pattern name of this strategy. + * @return the referenced pattern name. + */ + public String getPatternName() { + return patternName; + } + + @Override + public String toString() { + switch (strategy) { + case NO_SKIP: + case SKIP_PAST_LAST_EVENT: + return "AfterMatchStrategy{" + + strategy + + "}"; + case SKIP_TO_FIRST: + case SKIP_TO_LAST: + return "AfterMatchStrategy{" + + strategy + "[" + + patternName + "]" + + "}"; + } + return super.toString(); + } + + /** + * Skip Strategy Enum. + */ + public enum SkipStrategy{ + /** + * Every possible match will be emitted. + */ + NO_SKIP, + /** + * Discards every partial match that contains event of the match. + */ + SKIP_PAST_LAST_EVENT, + /** + * Discards every partial match that contains event of the match preceding the first of *PatternName*. + */ + SKIP_TO_FIRST, + /** + * Discards every partial match that contains event of the match preceding the last of *PatternName*. + */ + SKIP_TO_LAST + } +} diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java index 78ac39cac2ebd..ff4967fc55bec 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java @@ -55,6 +55,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -227,6 +228,26 @@ public void resetNFAChanged() { * activated) */ public Tuple2>>, Collection>, Long>>> process(final T event, final long timestamp) { + return process(event, timestamp, AfterMatchSkipStrategy.noSkip()); + } + + /** + * Processes the next input event. If some of the computations reach a final state then the + * resulting event sequences are returned. If computations time out and timeout handling is + * activated, then the timed out event patterns are returned. + * + *

If computations reach a stop state, the path forward is discarded and currently constructed path is returned + * with the element that resulted in the stop state. + * + * @param event The current event to be processed or null if only pruning shall be done + * @param timestamp The timestamp of the current event + * @param afterMatchSkipStrategy The skip strategy to use after per match + * @return Tuple of the collection of matched patterns (e.g. the result of computations which have + * reached a final state) and the collection of timed out patterns (if timeout handling is + * activated) + */ + public Tuple2>>, Collection>, Long>>> process(final T event, + final long timestamp, AfterMatchSkipStrategy afterMatchSkipStrategy) { final int numberComputationStates = computationStates.size(); final Collection>> result = new ArrayList<>(); final Collection>, Long>> timeoutResult = new ArrayList<>(); @@ -317,6 +338,8 @@ public Tuple2>>, Collection 0L) { long pruningTimestamp = timestamp - windowTime; @@ -335,6 +358,66 @@ public Tuple2>>, Collection> computationStates, + Collection>> matchedResult, AfterMatchSkipStrategy afterMatchSkipStrategy) { + Set discardEvents = new HashSet<>(); + switch(afterMatchSkipStrategy.getStrategy()) { + case SKIP_TO_LAST: + for (Map> resultMap: matchedResult) { + for (Map.Entry> keyMatches : resultMap.entrySet()) { + if (keyMatches.getKey().equals(afterMatchSkipStrategy.getPatternName())) { + discardEvents.addAll(keyMatches.getValue().subList(0, keyMatches.getValue().size() - 1)); + break; + } else { + discardEvents.addAll(keyMatches.getValue()); + } + } + } + break; + case SKIP_TO_FIRST: + for (Map> resultMap: matchedResult) { + for (Map.Entry> keyMatches : resultMap.entrySet()) { + if (keyMatches.getKey().equals(afterMatchSkipStrategy.getPatternName())) { + break; + } else { + discardEvents.addAll(keyMatches.getValue()); + } + } + } + break; + case SKIP_PAST_LAST_EVENT: + for (Map> resultMap: matchedResult) { + for (List eventList: resultMap.values()) { + discardEvents.addAll(eventList); + } + } + break; + } + if (!discardEvents.isEmpty()) { + List> discardStates = new ArrayList<>(); + for (ComputationState computationState : computationStates) { + Map> partialMatch = extractCurrentMatches(computationState); + for (List list: partialMatch.values()) { + for (T e: list) { + if (discardEvents.contains(e)) { + // discard the computation state. + eventSharedBuffer.release( + NFAStateNameHandler.getOriginalNameFromInternal( + computationState.getState().getName()), + computationState.getEvent(), + computationState.getTimestamp(), + computationState.getCounter() + ); + discardStates.add(computationState); + break; + } + } + } + } + computationStates.removeAll(discardStates); + } + } + @Override public boolean equals(Object obj) { if (obj instanceof NFA) { @@ -691,7 +774,7 @@ Map> extractCurrentMatches(final ComputationState computation // for a given computation state, we cannot have more than one matching patterns. Preconditions.checkState(paths.size() == 1); - Map> result = new HashMap<>(); + Map> result = new LinkedHashMap<>(); Map> path = paths.get(0); for (String key: path.keySet()) { List events = path.get(key); @@ -1035,11 +1118,11 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnaps return CompatibilityResult.compatible(); } else { if (eventCompatResult.getConvertDeserializer() != null && - sharedBufCompatResult.getConvertDeserializer() != null) { + sharedBufCompatResult.getConvertDeserializer() != null) { return CompatibilityResult.requiresMigration( - new NFASerializer<>( - new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()), - new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer()))); + new NFASerializer<>( + new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()), + new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer()))); } } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java index c36e7df866ce1..6bc5091eb8bd6 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java @@ -46,6 +46,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -247,7 +248,7 @@ public List>> extractPatterns( // termination criterion if (currentEntry == null) { - final Map> completePath = new HashMap<>(); + final Map> completePath = new LinkedHashMap<>(); while (!currentPath.isEmpty()) { final SharedBufferEntry currentPathEntry = currentPath.pop(); diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java index 5698de62d8b8d..39e8d34acef4d 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/compiler/NFACompiler.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.NFA; import org.apache.flink.cep.nfa.State; import org.apache.flink.cep.nfa.StateTransition; @@ -111,10 +112,12 @@ static class NFAFactoryCompiler { private Map, Boolean> firstOfLoopMap = new HashMap<>(); private Pattern currentPattern; private Pattern followingPattern; + private final AfterMatchSkipStrategy afterMatchSkipStrategy; private Map> originalStateMap = new HashMap<>(); NFAFactoryCompiler(final Pattern pattern) { this.currentPattern = pattern; + afterMatchSkipStrategy = pattern.getAfterMatchSkipStrategy(); } /** @@ -128,6 +131,8 @@ void compileFactory() { checkPatternNameUniqueness(); + checkPatternSkipStrategy(); + // we're traversing the pattern from the end to the beginning --> the first state is the final state State sinkState = createEndingState(); // add all the normal states @@ -136,6 +141,10 @@ void compileFactory() { createStartState(sinkState); } + AfterMatchSkipStrategy getAfterMatchSkipStrategy(){ + return afterMatchSkipStrategy; + } + List> getStates() { return states; } @@ -144,6 +153,25 @@ long getWindowTime() { return windowTime; } + /** + * Check pattern after match skip strategy. + */ + private void checkPatternSkipStrategy() { + if (afterMatchSkipStrategy.getStrategy() == AfterMatchSkipStrategy.SkipStrategy.SKIP_TO_FIRST || + afterMatchSkipStrategy.getStrategy() == AfterMatchSkipStrategy.SkipStrategy.SKIP_TO_LAST) { + Pattern pattern = currentPattern; + while (pattern.getPrevious() != null && !pattern.getName().equals(afterMatchSkipStrategy.getPatternName())) { + pattern = pattern.getPrevious(); + } + + // pattern name match check. + if (!pattern.getName().equals(afterMatchSkipStrategy.getPatternName())) { + throw new MalformedPatternException("The pattern name specified in AfterMatchSkipStrategy " + + "can not be found in the given Pattern"); + } + } + } + /** * Check if there are duplicate pattern names. If yes, it * throws a {@link MalformedPatternException}. diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java index 257d3e70e4569..ae2d7e4c9ad38 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.cep.EventComparator; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.NFA; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.runtime.state.StateInitializationContext; @@ -92,13 +93,14 @@ public abstract class AbstractKeyedCEPPatternOperator comparator; + protected final AfterMatchSkipStrategy afterMatchSkipStrategy; + public AbstractKeyedCEPPatternOperator( final TypeSerializer inputSerializer, final boolean isProcessingTime, - final TypeSerializer keySerializer, final NFACompiler.NFAFactory nfaFactory, - final boolean migratingFromOldKeyedOperator, final EventComparator comparator, + final AfterMatchSkipStrategy afterMatchSkipStrategy, final F function) { super(function); @@ -106,6 +108,12 @@ public AbstractKeyedCEPPatternOperator( this.isProcessingTime = Preconditions.checkNotNull(isProcessingTime); this.nfaFactory = Preconditions.checkNotNull(nfaFactory); this.comparator = comparator; + + if (afterMatchSkipStrategy == null) { + this.afterMatchSkipStrategy = AfterMatchSkipStrategy.noSkip(); + } else { + this.afterMatchSkipStrategy = afterMatchSkipStrategy; + } } @Override diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java index cef11e2505093..7c0c55d4b2b89 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CEPOperatorUtils.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.ByteSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.functions.NullByteKeySelector; import org.apache.flink.cep.EventComparator; @@ -29,6 +28,7 @@ import org.apache.flink.cep.PatternSelectFunction; import org.apache.flink.cep.PatternStream; import org.apache.flink.cep.PatternTimeoutFunction; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.cep.pattern.Pattern; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -62,20 +62,18 @@ public static SingleOutputStreamOperator createPatternStream( final TypeInformation outTypeInfo) { return createPatternStream(inputStream, pattern, outTypeInfo, false, comparator, new OperatorBuilder() { @Override - public OneInputStreamOperator build( + public OneInputStreamOperator build( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, - EventComparator comparator) { + EventComparator comparator, + AfterMatchSkipStrategy skipStrategy) { return new SelectCepOperator<>( inputSerializer, isProcessingTime, - keySerializer, nfaFactory, - migratingFromOldKeyedOperator, comparator, + skipStrategy, selectFunction ); } @@ -111,20 +109,18 @@ public static SingleOutputStreamOperator createPatternStream( final TypeInformation outTypeInfo) { return createPatternStream(inputStream, pattern, outTypeInfo, false, comparator, new OperatorBuilder() { @Override - public OneInputStreamOperator build( + public OneInputStreamOperator build( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, - EventComparator comparator) { + EventComparator comparator, + AfterMatchSkipStrategy skipStrategy) { return new FlatSelectCepOperator<>( inputSerializer, isProcessingTime, - keySerializer, nfaFactory, - migratingFromOldKeyedOperator, comparator, + skipStrategy, selectFunction ); } @@ -167,20 +163,18 @@ public static SingleOutputStreamOperator createTimeoutPat final PatternFlatTimeoutFunction timeoutFunction) { return createPatternStream(inputStream, pattern, outTypeInfo, true, comparator, new OperatorBuilder() { @Override - public OneInputStreamOperator build( + public OneInputStreamOperator build( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, - EventComparator comparator) { + EventComparator comparator, + AfterMatchSkipStrategy skipStrategy) { return new FlatSelectTimeoutCepOperator<>( inputSerializer, isProcessingTime, - keySerializer, nfaFactory, - migratingFromOldKeyedOperator, comparator, + skipStrategy, selectFunction, timeoutFunction, outputTag @@ -225,20 +219,18 @@ public static SingleOutputStreamOperator createTimeoutPat final PatternTimeoutFunction timeoutFunction) { return createPatternStream(inputStream, pattern, outTypeInfo, true, comparator, new OperatorBuilder() { @Override - public OneInputStreamOperator build( + public OneInputStreamOperator build( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, - EventComparator comparator) { + EventComparator comparator, + AfterMatchSkipStrategy skipStrategy) { return new SelectTimeoutCepOperator<>( inputSerializer, isProcessingTime, - keySerializer, nfaFactory, - migratingFromOldKeyedOperator, comparator, + skipStrategy, selectFunction, timeoutFunction, outputTag @@ -277,21 +269,17 @@ private static SingleOutputStreamOperator createPatternStream( if (inputStream instanceof KeyedStream) { KeyedStream keyedStream = (KeyedStream) inputStream; - TypeSerializer keySerializer = keyedStream.getKeyType().createSerializer(keyedStream.getExecutionConfig()); - patternStream = keyedStream.transform( operatorBuilder.getKeyedOperatorName(), outTypeInfo, operatorBuilder.build( inputSerializer, isProcessingTime, - keySerializer, nfaFactory, - true, - comparator)); + comparator, + pattern.getAfterMatchSkipStrategy())); } else { KeySelector keySelector = new NullByteKeySelector<>(); - TypeSerializer keySerializer = ByteSerializer.INSTANCE; patternStream = inputStream.keyBy(keySelector).transform( operatorBuilder.getOperatorName(), @@ -299,10 +287,9 @@ private static SingleOutputStreamOperator createPatternStream( operatorBuilder.build( inputSerializer, isProcessingTime, - keySerializer, nfaFactory, - false, - comparator + comparator, + pattern.getAfterMatchSkipStrategy() )).forceNonParallel(); } @@ -310,13 +297,12 @@ private static SingleOutputStreamOperator createPatternStream( } private interface OperatorBuilder { - OneInputStreamOperator build( + OneInputStreamOperator build( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, - EventComparator comparator); + EventComparator comparator, + AfterMatchSkipStrategy skipStrategy); String getKeyedOperatorName(); diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/FlatSelectCepOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/FlatSelectCepOperator.java index 192a38b0e0351..5e493728299a7 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/FlatSelectCepOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/FlatSelectCepOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cep.EventComparator; import org.apache.flink.cep.PatternFlatSelectFunction; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.streaming.api.operators.TimestampedCollector; @@ -41,12 +42,11 @@ public class FlatSelectCepOperator public FlatSelectCepOperator( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, EventComparator comparator, + AfterMatchSkipStrategy skipStrategy, PatternFlatSelectFunction function) { - super(inputSerializer, isProcessingTime, keySerializer, nfaFactory, migratingFromOldKeyedOperator, comparator, function); + super(inputSerializer, isProcessingTime, nfaFactory, comparator, skipStrategy, function); } private transient TimestampedCollector collector; diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/FlatSelectTimeoutCepOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/FlatSelectTimeoutCepOperator.java index 58a9d53aafd17..4423bb1dd40bd 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/FlatSelectTimeoutCepOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/FlatSelectTimeoutCepOperator.java @@ -26,6 +26,7 @@ import org.apache.flink.cep.EventComparator; import org.apache.flink.cep.PatternFlatSelectFunction; import org.apache.flink.cep.PatternFlatTimeoutFunction; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.util.OutputTag; @@ -55,20 +56,18 @@ public class FlatSelectTimeoutCepOperator extends public FlatSelectTimeoutCepOperator( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, EventComparator comparator, + AfterMatchSkipStrategy skipStrategy, PatternFlatSelectFunction flatSelectFunction, PatternFlatTimeoutFunction flatTimeoutFunction, OutputTag outputTag) { super( inputSerializer, isProcessingTime, - keySerializer, nfaFactory, - migratingFromOldKeyedOperator, comparator, + skipStrategy, new FlatSelectWrapper<>(flatSelectFunction, flatTimeoutFunction)); this.timedOutOutputTag = outputTag; } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/SelectCepOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/SelectCepOperator.java index acd3cd3f91acc..cbb49e676f984 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/SelectCepOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/SelectCepOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cep.EventComparator; import org.apache.flink.cep.PatternSelectFunction; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -39,12 +40,11 @@ public class SelectCepOperator public SelectCepOperator( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, EventComparator comparator, + AfterMatchSkipStrategy skipStrategy, PatternSelectFunction function) { - super(inputSerializer, isProcessingTime, keySerializer, nfaFactory, migratingFromOldKeyedOperator, comparator, function); + super(inputSerializer, isProcessingTime, nfaFactory, comparator, skipStrategy, function); } @Override diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/SelectTimeoutCepOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/SelectTimeoutCepOperator.java index d03e25c64e377..cb233a486ec5e 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/SelectTimeoutCepOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/SelectTimeoutCepOperator.java @@ -25,6 +25,7 @@ import org.apache.flink.cep.EventComparator; import org.apache.flink.cep.PatternSelectFunction; import org.apache.flink.cep.PatternTimeoutFunction; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.compiler.NFACompiler; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.util.OutputTag; @@ -50,20 +51,18 @@ public class SelectTimeoutCepOperator public SelectTimeoutCepOperator( TypeSerializer inputSerializer, boolean isProcessingTime, - TypeSerializer keySerializer, NFACompiler.NFAFactory nfaFactory, - boolean migratingFromOldKeyedOperator, final EventComparator comparator, + AfterMatchSkipStrategy skipStrategy, PatternSelectFunction flatSelectFunction, PatternTimeoutFunction flatTimeoutFunction, OutputTag outputTag) { super( inputSerializer, isProcessingTime, - keySerializer, nfaFactory, - migratingFromOldKeyedOperator, comparator, + skipStrategy, new SelectWrapper<>(flatSelectFunction, flatTimeoutFunction)); this.timedOutOutputTag = outputTag; } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/GroupPattern.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/GroupPattern.java index a20d37795c46e..fce408ce6e9ef 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/GroupPattern.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/GroupPattern.java @@ -18,6 +18,7 @@ package org.apache.flink.cep.pattern; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.pattern.conditions.IterativeCondition; /** @@ -31,16 +32,12 @@ public class GroupPattern extends Pattern { /** Group pattern representing the pattern definition of this group. */ private final Pattern groupPattern; - GroupPattern(final Pattern previous, final Pattern groupPattern) { - super("GroupPattern", previous); - this.groupPattern = groupPattern; - } - GroupPattern( final Pattern previous, final Pattern groupPattern, - final Quantifier.ConsumingStrategy consumingStrategy) { - super("GroupPattern", previous, consumingStrategy); + final Quantifier.ConsumingStrategy consumingStrategy, + final AfterMatchSkipStrategy afterMatchSkipStrategy) { + super("GroupPattern", previous, consumingStrategy, afterMatchSkipStrategy); this.groupPattern = groupPattern; } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java index 33574b39d249c..a276d9a5d3a46 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/Pattern.java @@ -19,6 +19,7 @@ package org.apache.flink.cep.pattern; import org.apache.flink.api.java.ClosureCleaner; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.NFA; import org.apache.flink.cep.pattern.Quantifier.ConsumingStrategy; import org.apache.flink.cep.pattern.Quantifier.Times; @@ -70,18 +71,17 @@ public class Pattern { */ private Times times; - protected Pattern(final String name, final Pattern previous) { - this.name = name; - this.previous = previous; - } + private final AfterMatchSkipStrategy afterMatchSkipStrategy; protected Pattern( - final String name, - final Pattern previous, - final ConsumingStrategy consumingStrategy) { + final String name, + final Pattern previous, + final ConsumingStrategy consumingStrategy, + final AfterMatchSkipStrategy afterMatchSkipStrategy) { this.name = name; this.previous = previous; this.quantifier = Quantifier.one(consumingStrategy); + this.afterMatchSkipStrategy = afterMatchSkipStrategy; } public Pattern getPrevious() { @@ -121,7 +121,20 @@ public IterativeCondition getUntilCondition() { * @return The first pattern of a pattern sequence */ public static Pattern begin(final String name) { - return new Pattern(name, null); + return new Pattern<>(name, null, ConsumingStrategy.STRICT, AfterMatchSkipStrategy.noSkip()); + } + + /** + * Starts a new pattern sequence. The provided name is the one of the initial pattern + * of the new sequence. Furthermore, the base type of the event sequence is set. + * + * @param name The name of starting pattern of the new pattern sequence + * @param afterMatchSkipStrategy the {@link AfterMatchSkipStrategy.SkipStrategy} to use after each match. + * @param Base type of the event pattern + * @return The first pattern of a pattern sequence + */ + public static Pattern begin(final String name, final AfterMatchSkipStrategy afterMatchSkipStrategy) { + return new Pattern(name, null, ConsumingStrategy.STRICT, afterMatchSkipStrategy); } /** @@ -241,7 +254,7 @@ public Pattern within(Time windowTime) { * @return A new pattern which is appended to this one */ public Pattern next(final String name) { - return new Pattern<>(name, this, ConsumingStrategy.STRICT); + return new Pattern<>(name, this, ConsumingStrategy.STRICT, afterMatchSkipStrategy); } /** @@ -258,7 +271,7 @@ public Pattern notNext(final String name) { "You can simulate such pattern with two independent patterns, one with and the other without " + "the optional part."); } - return new Pattern<>(name, this, ConsumingStrategy.NOT_NEXT); + return new Pattern<>(name, this, ConsumingStrategy.NOT_NEXT, afterMatchSkipStrategy); } /** @@ -270,7 +283,7 @@ public Pattern notNext(final String name) { * @return A new pattern which is appended to this one */ public Pattern followedBy(final String name) { - return new Pattern<>(name, this, ConsumingStrategy.SKIP_TILL_NEXT); + return new Pattern<>(name, this, ConsumingStrategy.SKIP_TILL_NEXT, afterMatchSkipStrategy); } /** @@ -289,7 +302,7 @@ public Pattern notFollowedBy(final String name) { "You can simulate such pattern with two independent patterns, one with and the other without " + "the optional part."); } - return new Pattern<>(name, this, ConsumingStrategy.NOT_FOLLOW); + return new Pattern<>(name, this, ConsumingStrategy.NOT_FOLLOW, afterMatchSkipStrategy); } /** @@ -301,7 +314,7 @@ public Pattern notFollowedBy(final String name) { * @return A new pattern which is appended to this one */ public Pattern followedByAny(final String name) { - return new Pattern<>(name, this, ConsumingStrategy.SKIP_TILL_ANY); + return new Pattern<>(name, this, ConsumingStrategy.SKIP_TILL_ANY, afterMatchSkipStrategy); } /** @@ -463,6 +476,19 @@ public Pattern consecutive() { return this; } + /** + * Starts a new pattern sequence. The provided pattern is the initial pattern + * of the new sequence. + * + * + * @param group the pattern to begin with + * @param afterMatchSkipStrategy the {@link AfterMatchSkipStrategy.SkipStrategy} to use after each match. + * @return The first pattern of a pattern sequence + */ + public static GroupPattern begin(final Pattern group, final AfterMatchSkipStrategy afterMatchSkipStrategy) { + return new GroupPattern<>(null, group, ConsumingStrategy.STRICT, afterMatchSkipStrategy); + } + /** * Starts a new pattern sequence. The provided pattern is the initial pattern * of the new sequence. @@ -471,7 +497,7 @@ public Pattern consecutive() { * @return the first pattern of a pattern sequence */ public static GroupPattern begin(Pattern group) { - return new GroupPattern<>(null, group); + return new GroupPattern<>(null, group, ConsumingStrategy.STRICT, AfterMatchSkipStrategy.noSkip()); } /** @@ -483,7 +509,7 @@ public static GroupPattern begin(Pattern group) { * @return A new pattern which is appended to this one */ public GroupPattern followedBy(Pattern group) { - return new GroupPattern<>(this, group, ConsumingStrategy.SKIP_TILL_NEXT); + return new GroupPattern<>(this, group, ConsumingStrategy.SKIP_TILL_NEXT, afterMatchSkipStrategy); } /** @@ -495,7 +521,7 @@ public GroupPattern followedBy(Pattern group) { * @return A new pattern which is appended to this one */ public GroupPattern followedByAny(Pattern group) { - return new GroupPattern<>(this, group, ConsumingStrategy.SKIP_TILL_ANY); + return new GroupPattern<>(this, group, ConsumingStrategy.SKIP_TILL_ANY, afterMatchSkipStrategy); } /** @@ -508,7 +534,7 @@ public GroupPattern followedByAny(Pattern group) { * @return A new pattern which is appended to this one */ public GroupPattern next(Pattern group) { - return new GroupPattern<>(this, group, ConsumingStrategy.STRICT); + return new GroupPattern<>(this, group, ConsumingStrategy.STRICT, afterMatchSkipStrategy); } private void checkIfNoNotPattern() { @@ -525,6 +551,13 @@ private void checkIfQuantifierApplied() { } } + /** + * @return the pattern's {@link AfterMatchSkipStrategy.SkipStrategy} after match. + */ + public AfterMatchSkipStrategy getAfterMatchSkipStrategy() { + return afterMatchSkipStrategy; + } + private void checkIfNoGroupPattern() { if (this instanceof GroupPattern) { throw new MalformedPatternException("Option not applicable to group pattern"); diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java new file mode 100644 index 0000000000000..f767d92459ebb --- /dev/null +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/AfterMatchSkipITCase.java @@ -0,0 +1,484 @@ +/* + * 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.cep.nfa; + +import org.apache.flink.cep.Event; +import org.apache.flink.cep.nfa.compiler.NFACompiler; +import org.apache.flink.cep.pattern.Pattern; +import org.apache.flink.cep.pattern.conditions.SimpleCondition; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.cep.nfa.NFATestUtilities.compareMaps; +import static org.apache.flink.cep.nfa.NFATestUtilities.feedNFA; + +/** + * IT tests covering {@link AfterMatchSkipStrategy}. + */ +public class AfterMatchSkipITCase extends TestLogger{ + + @Test + public void testSkipToNext() { + List> streamEvents = new ArrayList<>(); + + Event a1 = new Event(1, "a", 0.0); + Event a2 = new Event(2, "a", 0.0); + Event a3 = new Event(3, "a", 0.0); + Event a4 = new Event(4, "a", 0.0); + Event a5 = new Event(5, "a", 0.0); + Event a6 = new Event(6, "a", 0.0); + + streamEvents.add(new StreamRecord(a1)); + streamEvents.add(new StreamRecord(a2)); + streamEvents.add(new StreamRecord(a3)); + streamEvents.add(new StreamRecord(a4)); + streamEvents.add(new StreamRecord(a5)); + streamEvents.add(new StreamRecord(a6)); + + Pattern pattern = Pattern.begin("start", AfterMatchSkipStrategy.noSkip()) + .where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).times(3); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(a1, a2, a3), + Lists.newArrayList(a2, a3, a4), + Lists.newArrayList(a3, a4, a5), + Lists.newArrayList(a4, a5, a6) + )); + } + + @Test + public void testSkipPastLast() { + List> streamEvents = new ArrayList<>(); + + Event a1 = new Event(1, "a", 0.0); + Event a2 = new Event(2, "a", 0.0); + Event a3 = new Event(3, "a", 0.0); + Event a4 = new Event(4, "a", 0.0); + Event a5 = new Event(5, "a", 0.0); + Event a6 = new Event(6, "a", 0.0); + + streamEvents.add(new StreamRecord(a1)); + streamEvents.add(new StreamRecord(a2)); + streamEvents.add(new StreamRecord(a3)); + streamEvents.add(new StreamRecord(a4)); + streamEvents.add(new StreamRecord(a5)); + streamEvents.add(new StreamRecord(a6)); + + Pattern pattern = Pattern.begin("start", AfterMatchSkipStrategy.skipPastLastEvent()) + .where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().equals("a"); + } + }).times(3); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(a1, a2, a3), + Lists.newArrayList(a4, a5, a6) + )); + } + + @Test + public void testSkipToFirst() { + List> streamEvents = new ArrayList<>(); + + Event ab1 = new Event(1, "ab", 0.0); + Event ab2 = new Event(2, "ab", 0.0); + Event ab3 = new Event(3, "ab", 0.0); + Event ab4 = new Event(4, "ab", 0.0); + Event ab5 = new Event(5, "ab", 0.0); + Event ab6 = new Event(6, "ab", 0.0); + + streamEvents.add(new StreamRecord(ab1)); + streamEvents.add(new StreamRecord(ab2)); + streamEvents.add(new StreamRecord(ab3)); + streamEvents.add(new StreamRecord(ab4)); + streamEvents.add(new StreamRecord(ab5)); + streamEvents.add(new StreamRecord(ab6)); + + Pattern pattern = Pattern.begin("start", + AfterMatchSkipStrategy.skipToFirst("end")) + .where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("a"); + } + }).times(2).next("end").where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("b"); + } + }).times(2); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(ab1, ab2, ab3, ab4), + Lists.newArrayList(ab3, ab4, ab5, ab6) + )); + } + + @Test + public void testSkipToLast() { + List> streamEvents = new ArrayList<>(); + + Event ab1 = new Event(1, "ab", 0.0); + Event ab2 = new Event(2, "ab", 0.0); + Event ab3 = new Event(3, "ab", 0.0); + Event ab4 = new Event(4, "ab", 0.0); + Event ab5 = new Event(5, "ab", 0.0); + Event ab6 = new Event(6, "ab", 0.0); + Event ab7 = new Event(7, "ab", 0.0); + + streamEvents.add(new StreamRecord(ab1)); + streamEvents.add(new StreamRecord(ab2)); + streamEvents.add(new StreamRecord(ab3)); + streamEvents.add(new StreamRecord(ab4)); + streamEvents.add(new StreamRecord(ab5)); + streamEvents.add(new StreamRecord(ab6)); + streamEvents.add(new StreamRecord(ab7)); + + Pattern pattern = Pattern.begin("start", AfterMatchSkipStrategy.skipToLast("end")).where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("a"); + } + }).times(2).next("end").where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("b"); + } + }).times(2); + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(ab1, ab2, ab3, ab4), + Lists.newArrayList(ab4, ab5, ab6, ab7) + )); + } + + @Test + public void testSkipPastLast2() { + List> streamEvents = new ArrayList<>(); + + Event a1 = new Event(1, "a1", 0.0); + Event a2 = new Event(2, "a2", 0.0); + Event b1 = new Event(3, "b1", 0.0); + Event b2 = new Event(4, "b2", 0.0); + Event c1 = new Event(5, "c1", 0.0); + Event c2 = new Event(6, "c2", 0.0); + Event d1 = new Event(7, "d1", 0.0); + Event d2 = new Event(7, "d2", 0.0); + + streamEvents.add(new StreamRecord(a1)); + streamEvents.add(new StreamRecord(a2)); + streamEvents.add(new StreamRecord(b1)); + streamEvents.add(new StreamRecord(b2)); + streamEvents.add(new StreamRecord(c1)); + streamEvents.add(new StreamRecord(c2)); + streamEvents.add(new StreamRecord(d1)); + streamEvents.add(new StreamRecord(d2)); + + Pattern pattern = Pattern.begin("a", AfterMatchSkipStrategy.skipPastLastEvent()).where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("a"); + } + }).followedByAny("b").where( + new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("b"); + } + } + ).followedByAny("c").where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("c"); + } + }).followedByAny("d").where(new SimpleCondition() { + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("d"); + } + }); + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(a1, b1, c1, d1), + Lists.newArrayList(a1, b1, c2, d1), + Lists.newArrayList(a1, b2, c1, d1), + Lists.newArrayList(a1, b2, c2, d1), + Lists.newArrayList(a2, b1, c1, d1), + Lists.newArrayList(a2, b1, c2, d1), + Lists.newArrayList(a2, b2, c1, d1), + Lists.newArrayList(a2, b2, c2, d1) + )); + } + + @Test + public void testSkipPastLast3() { + List> streamEvents = new ArrayList<>(); + + Event a1 = new Event(1, "a1", 0.0); + Event c = new Event(2, "c", 0.0); + Event a2 = new Event(3, "a2", 0.0); + Event b2 = new Event(4, "b2", 0.0); + + streamEvents.add(new StreamRecord(a1)); + streamEvents.add(new StreamRecord(c)); + streamEvents.add(new StreamRecord(a2)); + streamEvents.add(new StreamRecord(b2)); + + Pattern pattern = Pattern.begin("a", AfterMatchSkipStrategy.skipPastLastEvent() + ).where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("a"); + } + }).next("b").where( + new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("b"); + } + } + ); + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.>newArrayList( + Lists.newArrayList(a2, b2) + )); + } + + @Test + public void testSkipToFirstWithOptionalMatch() { + List> streamEvents = new ArrayList<>(); + + Event ab1 = new Event(1, "ab1", 0.0); + Event c1 = new Event(2, "c1", 0.0); + Event ab2 = new Event(3, "ab2", 0.0); + Event c2 = new Event(4, "c2", 0.0); + + streamEvents.add(new StreamRecord(ab1)); + streamEvents.add(new StreamRecord(c1)); + streamEvents.add(new StreamRecord(ab2)); + streamEvents.add(new StreamRecord(c2)); + + Pattern pattern = Pattern.begin("x", AfterMatchSkipStrategy.skipToFirst("b") + ).where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("x"); + } + }).oneOrMore().optional().next("b").where( + new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("b"); + } + } + ).next("c").where(new SimpleCondition() { + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("c"); + } + }); + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(ab1, c1), + Lists.newArrayList(ab2, c2) + )); + } + + @Test + public void testSkipToFirstAtStartPosition() { + List> streamEvents = new ArrayList<>(); + + Event ab1 = new Event(1, "ab1", 0.0); + Event c1 = new Event(2, "c1", 0.0); + Event ab2 = new Event(3, "ab2", 0.0); + Event c2 = new Event(4, "c2", 0.0); + + streamEvents.add(new StreamRecord(ab1)); + streamEvents.add(new StreamRecord(c1)); + streamEvents.add(new StreamRecord(ab2)); + streamEvents.add(new StreamRecord(c2)); + + Pattern pattern = Pattern.begin("b", AfterMatchSkipStrategy.skipToFirst("b") + ).where( + new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("b"); + } + } + ).next("c").where(new SimpleCondition() { + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("c"); + } + }); + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(ab1, c1), + Lists.newArrayList(ab2, c2) + )); + } + + @Test + public void testSkipToFirstWithOneOrMore() { + List> streamEvents = new ArrayList<>(); + + Event a1 = new Event(1, "a1", 0.0); + Event b1 = new Event(2, "b1", 0.0); + Event a2 = new Event(3, "a2", 0.0); + Event b2 = new Event(4, "b2", 0.0); + Event b3 = new Event(5, "b3", 0.0); + Event a3 = new Event(3, "a3", 0.0); + Event b4 = new Event(4, "b4", 0.0); + + streamEvents.add(new StreamRecord(a1)); + streamEvents.add(new StreamRecord(b1)); + streamEvents.add(new StreamRecord(a2)); + streamEvents.add(new StreamRecord(b2)); + streamEvents.add(new StreamRecord(b3)); + streamEvents.add(new StreamRecord(a3)); + streamEvents.add(new StreamRecord(b4)); + + Pattern pattern = Pattern.begin("a", AfterMatchSkipStrategy.skipToFirst("b") + ).where( + new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("a"); + } + } + ).next("b").where(new SimpleCondition() { + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("b"); + } + }).oneOrMore().consecutive(); + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(a1, b1), + Lists.newArrayList(a2, b2), + Lists.newArrayList(a3, b4) + )); + } + + @Test + public void testSkipToLastWithOneOrMore() { + List> streamEvents = new ArrayList<>(); + + Event a1 = new Event(1, "a1", 0.0); + Event b1 = new Event(2, "b1", 0.0); + Event a2 = new Event(3, "a2", 0.0); + Event b2 = new Event(4, "b2", 0.0); + Event b3 = new Event(5, "b3", 0.0); + Event a3 = new Event(3, "a3", 0.0); + Event b4 = new Event(4, "b4", 0.0); + + streamEvents.add(new StreamRecord(a1)); + streamEvents.add(new StreamRecord(b1)); + streamEvents.add(new StreamRecord(a2)); + streamEvents.add(new StreamRecord(b2)); + streamEvents.add(new StreamRecord(b3)); + streamEvents.add(new StreamRecord(a3)); + streamEvents.add(new StreamRecord(b4)); + + Pattern pattern = Pattern.begin("a", AfterMatchSkipStrategy.skipToLast("b") + ).where( + new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("a"); + } + } + ).next("b").where(new SimpleCondition() { + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("b"); + } + }).oneOrMore().consecutive(); + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List> resultingPatterns = feedNFA(streamEvents, nfa, pattern.getAfterMatchSkipStrategy()); + + compareMaps(resultingPatterns, Lists.newArrayList( + Lists.newArrayList(a1, b1), + Lists.newArrayList(a2, b2), + Lists.newArrayList(a3, b4) + )); + } +} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java index b7798c58f2c3e..84278b15fd462 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFAITCase.java @@ -2722,4 +2722,76 @@ public boolean filter(Event s) throws Exception { match.get("middle").toArray(), Lists.newArrayList(endEvent1, endEvent2, endEvent3).toArray()); } + + @Test + public void testNFAResultKeyOrdering() { + List> inputEvents = new ArrayList<>(); + + Event a1 = new Event(41, "a", 2.0); + Event b1 = new Event(41, "b", 3.0); + Event aa1 = new Event(41, "aa", 4.0); + Event bb1 = new Event(41, "bb", 5.0); + Event ab1 = new Event(41, "ab", 6.0); + + inputEvents.add(new StreamRecord<>(a1, 1)); + inputEvents.add(new StreamRecord<>(b1, 3)); + inputEvents.add(new StreamRecord<>(aa1, 4)); + inputEvents.add(new StreamRecord<>(bb1, 5)); + inputEvents.add(new StreamRecord<>(ab1, 6)); + + Pattern pattern = Pattern + .begin("a") + .where(new SimpleCondition() { + private static final long serialVersionUID = 6452194090480345053L; + + @Override + public boolean filter(Event s) throws Exception { + return s.getName().equals("a"); + } + }).next("b").where(new SimpleCondition() { + @Override + public boolean filter(Event s) throws Exception { + return s.getName().equals("b"); + } + }).next("aa").where(new SimpleCondition() { + @Override + public boolean filter(Event s) throws Exception { + return s.getName().equals("aa"); + } + }).next("bb").where(new SimpleCondition() { + @Override + public boolean filter(Event s) throws Exception { + return s.getName().equals("bb"); + } + }).next("ab").where(new SimpleCondition() { + @Override + public boolean filter(Event s) throws Exception { + return s.getName().equals("ab"); + } + }); + + NFA nfa = NFACompiler.compile(pattern, Event.createTypeSerializer(), false); + + List>> resultingPatterns = new ArrayList<>(); + + for (StreamRecord inputEvent : inputEvents) { + Collection>> patterns = nfa.process( + inputEvent.getValue(), + inputEvent.getTimestamp()).f0; + + resultingPatterns.addAll(patterns); + } + + Assert.assertEquals(1L, resultingPatterns.size()); + + Map> match = resultingPatterns.get(0); + + List expectedOrder = Lists.newArrayList("a", "b", "aa", "bb", "ab"); + List resultOrder = new ArrayList<>(); + for (String key: match.keySet()) { + resultOrder.add(key); + } + + Assert.assertEquals(expectedOrder, resultOrder); + } } diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java index 7a522ef1bbf7f..a9e17955ac8ba 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/NFATestUtilities.java @@ -36,12 +36,18 @@ public class NFATestUtilities { public static List> feedNFA(List> inputEvents, NFA nfa) { + return feedNFA(inputEvents, nfa, AfterMatchSkipStrategy.noSkip()); + } + + public static List> feedNFA(List> inputEvents, NFA nfa, + AfterMatchSkipStrategy afterMatchSkipStrategy) { List> resultingPatterns = new ArrayList<>(); for (StreamRecord inputEvent : inputEvents) { Collection>> patterns = nfa.process( inputEvent.getValue(), - inputEvent.getTimestamp()).f0; + inputEvent.getTimestamp(), + afterMatchSkipStrategy).f0; for (Map> p: patterns) { List res = new ArrayList<>(); diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java index 3621bade5884e..dfbfa5fcd3f72 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/SharedBufferTest.java @@ -33,6 +33,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -190,4 +191,49 @@ public void testClearingSharedBufferWithMultipleEdgesBetweenEntries() { //There should be still events[1] and events[2] in the buffer assertFalse(sharedBuffer.isEmpty()); } + + @Test + public void testSharedBufferExtractOrder() { + SharedBuffer sharedBuffer = new SharedBuffer<>(Event.createTypeSerializer()); + int numberEvents = 10; + Event[] events = new Event[numberEvents]; + final long timestamp = 1L; + + for (int i = 0; i < numberEvents; i++) { + events[i] = new Event(i + 1, "e" + (i + 1), i); + } + + Map> expectedResult = new LinkedHashMap<>(); + expectedResult.put("a", new ArrayList<>()); + expectedResult.get("a").add(events[1]); + expectedResult.put("b", new ArrayList<>()); + expectedResult.get("b").add(events[2]); + expectedResult.put("aa", new ArrayList<>()); + expectedResult.get("aa").add(events[3]); + expectedResult.put("bb", new ArrayList<>()); + expectedResult.get("bb").add(events[4]); + expectedResult.put("c", new ArrayList<>()); + expectedResult.get("c").add(events[5]); + + sharedBuffer.put("a", events[1], timestamp, DeweyNumber.fromString("1")); + sharedBuffer.put("b", events[2], timestamp, "a", events[1], timestamp, 0, DeweyNumber.fromString("1.0")); + sharedBuffer.put("aa", events[3], timestamp, "b", events[2], timestamp, 1, DeweyNumber.fromString("1.0.0")); + sharedBuffer.put("bb", events[4], timestamp, "aa", events[3], timestamp, 2, DeweyNumber.fromString("1.0.0.0")); + sharedBuffer.put("c", events[5], timestamp, "bb", events[4], timestamp, 3, DeweyNumber.fromString("1.0.0.0.0")); + + Collection>> patternsResult = sharedBuffer.extractPatterns("c", events[5], timestamp, 4, DeweyNumber.fromString("1.0.0.0.0")); + + List expectedOrder = new ArrayList<>(); + expectedOrder.add("a"); + expectedOrder.add("b"); + expectedOrder.add("aa"); + expectedOrder.add("bb"); + expectedOrder.add("c"); + + List resultOrder = new ArrayList<>(); + for (String key: patternsResult.iterator().next().keySet()){ + resultOrder.add(key); + } + assertEquals(expectedOrder, resultOrder); + } } diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java index 13dc6da927fb6..ec2cf47578cf6 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/nfa/compiler/NFACompilerTest.java @@ -24,6 +24,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.cep.Event; import org.apache.flink.cep.SubEvent; +import org.apache.flink.cep.nfa.AfterMatchSkipStrategy; import org.apache.flink.cep.nfa.NFA; import org.apache.flink.cep.nfa.State; import org.apache.flink.cep.nfa.StateTransition; @@ -188,6 +189,37 @@ public void testNoUnnecessaryStateCopiesCreated() { assertEquals(1, endStateCount); } + @Test + public void testSkipToNotExistsMatchingPattern() { + expectedException.expect(MalformedPatternException.class); + expectedException.expectMessage("The pattern name specified in AfterMatchSkipStrategy can not be found in the given Pattern"); + + Pattern invalidPattern = Pattern.begin("start", + AfterMatchSkipStrategy.skipToLast("midd")).where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("a"); + } + }).next("middle").where( + new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("d"); + } + } + ).oneOrMore().optional().next("end").where(new SimpleCondition() { + + @Override + public boolean filter(Event value) throws Exception { + return value.getName().contains("c"); + } + }); + + NFACompiler.compile(invalidPattern, Event.createTypeSerializer(), false); + } + private Set> unfoldTransitions(final State state) { final Set> transitions = new HashSet<>(); for (StateTransition transition : state.getStateTransitions()) { diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java index a2ac124276e56..9eb60de6f0981 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.ValueState; 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.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; @@ -256,9 +255,8 @@ public void testKeyedAdvancingTimeWithoutElements() throws Exception { new SelectTimeoutCepOperator<>( Event.createTypeSerializer(), false, - IntSerializer.INSTANCE, new NFAFactory(true), - true, + null, null, new PatternSelectFunction>>() { @Override diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java index 169028417ec69..f5236c1715d97 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPRescalingTest.java @@ -18,7 +18,6 @@ package org.apache.flink.cep.operator; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.cep.Event; @@ -375,8 +374,7 @@ private KeyedOneInputStreamOperatorTestHarness( getKeyedCepOpearator( false, - new NFAFactory(), - BasicTypeInfo.INT_TYPE_INFO.createSerializer(new ExecutionConfig())), + new NFAFactory()), keySelector, BasicTypeInfo.INT_TYPE_INFO, maxParallelism, diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java index 17d6656f79214..feb020ac7b45a 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepOperatorTestUtilities.java @@ -19,8 +19,6 @@ package org.apache.flink.cep.operator; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.cep.Event; import org.apache.flink.cep.EventComparator; @@ -57,49 +55,23 @@ public static OneInputStreamOperatorTestHarness>> BasicTypeInfo.INT_TYPE_INFO); } - public static SelectCepOperator>> getKeyedCepOpearator( - boolean isProcessingTime, - NFACompiler.NFAFactory nfaFactory) { - return getKeyedCepOpearator(isProcessingTime, nfaFactory, IntSerializer.INSTANCE, null); - } - - public static SelectCepOperator>> getKeyedCepOpearator( - boolean isProcessingTime, - NFACompiler.NFAFactory nfaFactory, - EventComparator comparator) { - return getKeyedCepOpearator(isProcessingTime, nfaFactory, IntSerializer.INSTANCE, comparator); - } - public static SelectCepOperator>> getKeyedCepOpearator( boolean isProcessingTime, - NFACompiler.NFAFactory nfaFactory, - TypeSerializer keySerializer, - EventComparator comparator) { - - return getKeyedCepOpearator(isProcessingTime, nfaFactory, keySerializer, true, comparator); - } - - public static SelectCepOperator>> getKeyedCepOpearator( - boolean isProcessingTime, - NFACompiler.NFAFactory nfaFactory, - TypeSerializer keySerializer) { + NFACompiler.NFAFactory nfaFactory) { - return getKeyedCepOpearator(isProcessingTime, nfaFactory, keySerializer, true, null); + return getKeyedCepOpearator(isProcessingTime, nfaFactory, null); } public static SelectCepOperator>> getKeyedCepOpearator( boolean isProcessingTime, NFACompiler.NFAFactory nfaFactory, - TypeSerializer keySerializer, - boolean migratingFromOldKeyedOperator, EventComparator comparator) { return new SelectCepOperator<>( Event.createTypeSerializer(), isProcessingTime, - keySerializer, nfaFactory, - migratingFromOldKeyedOperator, comparator, + null, new PatternSelectFunction>>() { @Override public Map> select(Map> pattern) throws Exception { From fb3bd1fceba6da362966491e55c3bf27566ede53 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 16 Aug 2017 14:36:13 +0200 Subject: [PATCH 077/129] [FLINK-7457] Make Dispatcher highly available This commit introduces a dispatcher leader election and retrieval service to the HighAvailabilityServices. Moreover it adds code such that the Dispatcher now takes part in the leader election process using the afore-mentioned services. Let Dispatcher participate in leader election Add test for Dispatcher leader election This closes #4548. --- .../flink/runtime/dispatcher/Dispatcher.java | 152 +++++++++++++++++- .../dispatcher/DispatcherException.java | 40 +++++ .../runtime/dispatcher/DispatcherGateway.java | 3 + .../HighAvailabilityServices.java | 13 ++ .../HighAvailabilityServicesUtils.java | 12 +- .../nonha/embedded/EmbeddedHaServices.java | 13 ++ .../standalone/StandaloneHaServices.java | 27 +++- .../zookeeper/ZooKeeperHaServices.java | 12 ++ .../apache/flink/runtime/rpc/RpcUtils.java | 22 +++ .../runtime/dispatcher/DispatcherTest.java | 114 +++++++++++-- .../TestingHighAvailabilityServices.java | 33 ++++ ...TestingManualHighAvailabilityServices.java | 13 ++ .../standalone/StandaloneHaServicesTest.java | 6 +- .../taskexecutor/TaskExecutorTest.java | 2 + .../YarnIntraNonHaMasterServices.java | 25 +++ .../YarnPreConfiguredMasterNonHaServices.java | 32 ++++ 16 files changed, 492 insertions(+), 27 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherException.java 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 bb0b3e4ab3cf4..29262cd973821 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 @@ -28,17 +28,21 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.LeaderIdMismatchException; import org.apache.flink.runtime.highavailability.RunningJobsRegistry; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmanager.SubmittedJobGraph; import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +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.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; @@ -47,6 +51,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; +import java.util.UUID; import java.util.concurrent.CompletableFuture; /** @@ -55,7 +60,7 @@ * the jobs and to recover them in case of a master failure. Furthermore, it knows * about the state of the Flink session cluster. */ -public abstract class Dispatcher extends RpcEndpoint implements DispatcherGateway { +public abstract class Dispatcher extends RpcEndpoint implements DispatcherGateway, LeaderContender { public static final String DISPATCHER_NAME = "dispatcher"; @@ -73,6 +78,10 @@ public abstract class Dispatcher extends RpcEndpoint implements DispatcherGatewa private final Map jobManagerRunners; + private final LeaderElectionService leaderElectionService; + + private volatile UUID leaderSessionId; + protected Dispatcher( RpcService rpcService, String endpointId, @@ -95,6 +104,11 @@ protected Dispatcher( this.runningJobsRegistry = highAvailabilityServices.getRunningJobsRegistry(); jobManagerRunners = new HashMap<>(16); + + leaderElectionService = highAvailabilityServices.getDispatcherLeaderElectionService(); + + // we are not the leader when this object is created + leaderSessionId = null; } //------------------------------------------------------ @@ -104,12 +118,8 @@ protected Dispatcher( @Override public void postStop() throws Exception { Exception exception = null; - // stop all currently running JobManagerRunners - for (JobManagerRunner jobManagerRunner : jobManagerRunners.values()) { - jobManagerRunner.shutdown(); - } - jobManagerRunners.clear(); + clearState(); try { submittedJobGraphStore.stop(); @@ -117,6 +127,12 @@ public void postStop() throws Exception { exception = ExceptionUtils.firstOrSuppressed(e, exception); } + try { + leaderElectionService.stop(); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); + } + try { super.postStop(); } catch (Exception e) { @@ -128,12 +144,26 @@ public void postStop() throws Exception { } } + @Override + public void start() throws Exception { + super.start(); + + leaderElectionService.start(this); + } + //------------------------------------------------------ // RPCs //------------------------------------------------------ @Override - public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) { + public CompletableFuture submitJob(JobGraph jobGraph, UUID leaderSessionId, Time timeout) { + + try { + validateLeaderSessionId(leaderSessionId); + } catch (LeaderIdMismatchException e) { + return FutureUtils.completedExceptionally(e); + } + final JobID jobId = jobGraph.getJobID(); log.info("Submitting job {} ({}).", jobGraph.getJobID(), jobGraph.getName()); @@ -224,6 +254,62 @@ private void removeJob(JobID jobId, boolean cleanupHA) throws Exception { // TODO: remove job related files from blob server } + /** + * Clears the state of the dispatcher. + * + *

The state are all currently running jobs. + */ + private void clearState() { + // stop all currently running JobManager since they run in the same process + for (JobManagerRunner jobManagerRunner : jobManagerRunners.values()) { + jobManagerRunner.shutdown(); + } + + jobManagerRunners.clear(); + } + + /** + * Recovers all jobs persisted via the submitted job graph store. + */ + private void recoverJobs() { + log.info("Recovering all persisted jobs."); + + final UUID currentLeaderSessionId = leaderSessionId; + + getRpcService().execute( + () -> { + final Collection jobIds; + + try { + jobIds = submittedJobGraphStore.getJobIds(); + } catch (Exception e) { + log.error("Could not recover job ids from the submitted job graph store. Aborting recovery.", e); + return; + } + + for (JobID jobId : jobIds) { + try { + SubmittedJobGraph submittedJobGraph = submittedJobGraphStore.recoverJobGraph(jobId); + + runAsync(() -> submitJob(submittedJobGraph.getJobGraph(), currentLeaderSessionId, RpcUtils.INF_TIMEOUT)); + } catch (Exception e) { + log.error("Could not recover the job graph for " + jobId + '.', e); + } + } + }); + } + + private void onFatalError(Throwable throwable) { + log.error("Fatal error occurred in dispatcher {}.", getAddress(), throwable); + fatalErrorHandler.onFatalError(throwable); + } + + private void validateLeaderSessionId(UUID leaderSessionID) throws LeaderIdMismatchException { + if (this.leaderSessionId == null || !this.leaderSessionId.equals(leaderSessionID)) { + throw new LeaderIdMismatchException(this.leaderSessionId, leaderSessionID); + } + } + protected abstract JobManagerRunner createJobManagerRunner( ResourceID resourceId, JobGraph jobGraph, @@ -236,6 +322,58 @@ protected abstract JobManagerRunner createJobManagerRunner( OnCompletionActions onCompleteActions, FatalErrorHandler fatalErrorHandler) throws Exception; + //------------------------------------------------------ + // Leader contender + //------------------------------------------------------ + + /** + * Callback method when current resourceManager is granted leadership. + * + * @param newLeaderSessionID unique leadershipID + */ + @Override + public void grantLeadership(final UUID newLeaderSessionID) { + runAsync( + () -> { + log.info("Dispatcher {} was granted leadership with leader session ID {}", getAddress(), newLeaderSessionID); + + // clear the state if we've been the leader before + if (leaderSessionId != null) { + clearState(); + } + + leaderSessionId = newLeaderSessionID; + + // confirming the leader session ID might be blocking, + getRpcService().execute( + () -> leaderElectionService.confirmLeaderSessionID(newLeaderSessionID)); + + recoverJobs(); + }); + } + + /** + * Callback method when current resourceManager loses leadership. + */ + @Override + public void revokeLeadership() { + runAsync( + () -> { + log.info("Dispatcher {} was revoked leadership.", getAddress()); + clearState(); + }); + } + + /** + * Handles error occurring in the leader election service. + * + * @param exception Exception being thrown in the leader election service + */ + @Override + public void handleError(final Exception exception) { + onFatalError(new DispatcherException("Received an error from the LeaderElectionService.", exception)); + } + //------------------------------------------------------ // Utility classes //------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherException.java new file mode 100644 index 0000000000000..cf4a49300a5ca --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherException.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.dispatcher; + +import org.apache.flink.util.FlinkException; + +/** + * Base class for {@link Dispatcher} related exceptions. + */ +public class DispatcherException extends FlinkException { + private static final long serialVersionUID = 3781733042984381286L; + + public DispatcherException(String message) { + super(message); + } + + public DispatcherException(Throwable cause) { + super(cause); + } + + public DispatcherException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java index 33b8a42a364d4..669f616a704db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.rpc.RpcTimeout; import java.util.Collection; +import java.util.UUID; import java.util.concurrent.CompletableFuture; /** @@ -37,11 +38,13 @@ public interface DispatcherGateway extends RpcGateway { * Submit a job to the dispatcher. * * @param jobGraph JobGraph to submit + * @param leaderSessionId leader session id * @param timeout RPC timeout * @return A future acknowledge if the submission succeeded */ CompletableFuture submitJob( JobGraph jobGraph, + UUID leaderSessionId, @RpcTimeout Time timeout); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index b44905e3d94c5..defe5cce331db 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -72,6 +72,12 @@ public interface HighAvailabilityServices extends AutoCloseable { */ LeaderRetrievalService getResourceManagerLeaderRetriever(); + /** + * Gets the leader retriever for the dispatcher. This leader retrieval service + * is not always accessible. + */ + LeaderRetrievalService getDispatcherLeaderRetriever(); + /** * Gets the leader retriever for the job JobMaster which is responsible for the given job * @@ -99,6 +105,13 @@ public interface HighAvailabilityServices extends AutoCloseable { */ LeaderElectionService getResourceManagerLeaderElectionService(); + /** + * Gets the leader election service for the cluster's dispatcher. + * + * @return Leader election service for the dispatcher leader election + */ + LeaderElectionService getDispatcherLeaderElectionService(); + /** * Gets the leader election service for the given job. * 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 2ebfd20245662..7a89ed8fdf898 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 @@ -23,6 +23,7 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.blob.BlobStoreService; import org.apache.flink.runtime.blob.BlobUtils; +import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServices; import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneHaServices; import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices; @@ -87,8 +88,17 @@ public static HighAvailabilityServices createHighAvailabilityServices( ResourceManager.RESOURCE_MANAGER_NAME, addressResolution, configuration); + final String dispatcherRpcUrl = AkkaRpcServiceUtils.getRpcUrl( + hostnamePort.f0, + hostnamePort.f1, + Dispatcher.DISPATCHER_NAME, + addressResolution, + configuration); - return new StandaloneHaServices(resourceManagerRpcUrl, jobManagerRpcUrl); + return new StandaloneHaServices( + resourceManagerRpcUrl, + dispatcherRpcUrl, + jobManagerRpcUrl); case ZOOKEEPER: BlobStoreService blobStoreService = BlobUtils.createBlobStoreFromConfig(configuration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServices.java index 76eb681e3e8e0..4c30f87fba7fc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServices.java @@ -45,11 +45,14 @@ public class EmbeddedHaServices extends AbstractNonHaServices { private final EmbeddedLeaderService resourceManagerLeaderService; + private final EmbeddedLeaderService dispatcherLeaderService; + private final HashMap jobManagerLeaderServices; public EmbeddedHaServices(Executor executor) { this.executor = Preconditions.checkNotNull(executor); this.resourceManagerLeaderService = new EmbeddedLeaderService(executor); + this.dispatcherLeaderService = new EmbeddedLeaderService(executor); this.jobManagerLeaderServices = new HashMap<>(); } @@ -62,11 +65,21 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() { return resourceManagerLeaderService.createLeaderRetrievalService(); } + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + return dispatcherLeaderService.createLeaderRetrievalService(); + } + @Override public LeaderElectionService getResourceManagerLeaderElectionService() { return resourceManagerLeaderService.createLeaderElectionService(); } + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + return dispatcherLeaderService.createLeaderElectionService(); + } + @Override public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { checkNotNull(jobID); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java index b3c6ee51b37bd..617b3512df611 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServices.java @@ -45,6 +45,9 @@ public class StandaloneHaServices extends AbstractNonHaServices { /** The fix address of the ResourceManager */ private final String resourceManagerAddress; + /** The fix address of the Dispatcher */ + private final String dispatcherAddress; + /** The fix address of the JobManager */ private final String jobManagerAddress; @@ -53,8 +56,12 @@ public class StandaloneHaServices extends AbstractNonHaServices { * * @param resourceManagerAddress The fix address of the ResourceManager */ - public StandaloneHaServices(String resourceManagerAddress, String jobManagerAddress) { + public StandaloneHaServices( + String resourceManagerAddress, + String dispatcherAddress, + String jobManagerAddress) { this.resourceManagerAddress = checkNotNull(resourceManagerAddress, "resourceManagerAddress"); + this.dispatcherAddress = checkNotNull(dispatcherAddress, "dispatcherAddress"); this.jobManagerAddress = checkNotNull(jobManagerAddress, "jobManagerAddress"); } @@ -72,6 +79,15 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() { } + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + synchronized (lock) { + checkNotShutdown(); + + return new StandaloneLeaderRetrievalService(dispatcherAddress, DEFAULT_LEADER_ID); + } + } + @Override public LeaderElectionService getResourceManagerLeaderElectionService() { synchronized (lock) { @@ -81,6 +97,15 @@ public LeaderElectionService getResourceManagerLeaderElectionService() { } } + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + synchronized (lock) { + checkNotShutdown(); + + return new StandaloneLeaderElectionService(); + } + } + @Override public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { synchronized (lock) { 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 9dabfa218284e..04ab6d3cc96e9 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 @@ -82,6 +82,8 @@ public class ZooKeeperHaServices implements HighAvailabilityServices { private static final String RESOURCE_MANAGER_LEADER_PATH = "/resource_manager_lock"; + private static final String DISPATCHER_LEADER_PATH = "/dispatcher_lock"; + private static final String JOB_MANAGER_LEADER_PATH = "/job_manager_lock"; // ------------------------------------------------------------------------ @@ -124,6 +126,11 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() { return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, RESOURCE_MANAGER_LEADER_PATH); } + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, DISPATCHER_LEADER_PATH); + } + @Override public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { return ZooKeeperUtils.createLeaderRetrievalService(client, configuration, getPathForJobManager(jobID)); @@ -139,6 +146,11 @@ public LeaderElectionService getResourceManagerLeaderElectionService() { return ZooKeeperUtils.createLeaderElectionService(client, configuration, RESOURCE_MANAGER_LEADER_PATH); } + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + return ZooKeeperUtils.createLeaderElectionService(client, configuration, DISPATCHER_LEADER_PATH); + } + @Override public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { return ZooKeeperUtils.createLeaderElectionService(client, configuration, getPathForJobManager(jobID)); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java index 9738970a61f5d..a644efd9853d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcUtils.java @@ -18,13 +18,21 @@ package org.apache.flink.runtime.rpc; +import org.apache.flink.api.common.time.Time; + import java.util.HashSet; import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; /** * Utility functions for Flink's RPC implementation */ public class RpcUtils { + + public static final Time INF_TIMEOUT = Time.milliseconds(Long.MAX_VALUE); + /** * Extracts all {@link RpcGateway} interfaces implemented by the given clazz. * @@ -47,6 +55,20 @@ public static Set> extractImplementedRpcGateways(Cla return interfaces; } + /** + * Shuts the given {@link RpcEndpoint} down and awaits its termination. + * + * @param rpcEndpoint to terminate + * @param timeout for this operation + * @throws ExecutionException if a problem occurs + * @throws InterruptedException if the operation has been interrupted + * @throws TimeoutException if a timeout occurred + */ + public static void terminateRpcEndpoint(RpcEndpoint rpcEndpoint, Time timeout) throws ExecutionException, InterruptedException, TimeoutException { + rpcEndpoint.shutDown(); + rpcEndpoint.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + // We don't want this class to be instantiable private RpcUtils() {} } 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 423732710100d..091608c1e4bcf 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,24 +25,33 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneHaServices; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; +import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.rpc.FatalErrorHandler; 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.util.TestingFatalErrorHandler; import org.apache.flink.util.TestLogger; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.junit.Test; import org.mockito.Mockito; +import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -52,6 +61,23 @@ */ public class DispatcherTest extends TestLogger { + private static RpcService rpcService; + private static final Time timeout = Time.seconds(10L); + + @BeforeClass + public static void setup() { + rpcService = new TestingRpcService(); + } + + @AfterClass + public static void teardown() { + if (rpcService != null) { + rpcService.stopService(); + + rpcService = null; + } + } + /** * Tests that we can submit a job to the Dispatcher which then spawns a * new JobManagerRunner. @@ -59,34 +85,35 @@ public class DispatcherTest extends TestLogger { @Test public void testJobSubmission() throws Exception { TestingFatalErrorHandler fatalErrorHandler = new TestingFatalErrorHandler(); - RpcService rpcService = new TestingRpcService(); - HighAvailabilityServices haServices = new StandaloneHaServices("localhost", "localhost"); + HighAvailabilityServices haServices = new StandaloneHaServices( + "localhost", + "localhost", + "localhost"); HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 10000L); JobManagerRunner jobManagerRunner = mock(JobManagerRunner.class); - final Time timeout = Time.seconds(5L); final JobGraph jobGraph = mock(JobGraph.class); final JobID jobId = new JobID(); when(jobGraph.getJobID()).thenReturn(jobId); - try { - final TestingDispatcher dispatcher = new TestingDispatcher( - rpcService, - Dispatcher.DISPATCHER_NAME, - new Configuration(), - haServices, - mock(BlobServer.class), - heartbeatServices, - mock(MetricRegistry.class), - fatalErrorHandler, - jobManagerRunner, - jobId); + final TestingDispatcher dispatcher = new TestingDispatcher( + rpcService, + Dispatcher.DISPATCHER_NAME, + new Configuration(), + haServices, + mock(BlobServer.class), + heartbeatServices, + mock(MetricRegistry.class), + fatalErrorHandler, + jobManagerRunner, + jobId); + try { dispatcher.start(); DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - CompletableFuture acknowledgeFuture = dispatcherGateway.submitJob(jobGraph, timeout); + CompletableFuture acknowledgeFuture = dispatcherGateway.submitJob(jobGraph, HighAvailabilityServices.DEFAULT_LEADER_ID, timeout); acknowledgeFuture.get(); @@ -95,7 +122,60 @@ public void testJobSubmission() throws Exception { // check that no error has occurred fatalErrorHandler.rethrowError(); } finally { - rpcService.stopService(); + RpcUtils.terminateRpcEndpoint(dispatcher, timeout); + } + } + + /** + * Tests that the dispatcher takes part in the leader election. + */ + @Test + public void testLeaderElection() throws Exception { + TestingFatalErrorHandler fatalErrorHandler = new TestingFatalErrorHandler(); + TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); + + UUID expectedLeaderSessionId = UUID.randomUUID(); + CompletableFuture leaderSessionIdFuture = new CompletableFuture<>(); + SubmittedJobGraphStore mockSubmittedJobGraphStore = mock(SubmittedJobGraphStore.class); + TestingLeaderElectionService testingLeaderElectionService = new TestingLeaderElectionService() { + @Override + public void confirmLeaderSessionID(UUID leaderSessionId) { + super.confirmLeaderSessionID(leaderSessionId); + leaderSessionIdFuture.complete(leaderSessionId); + } + }; + + haServices.setSubmittedJobGraphStore(mockSubmittedJobGraphStore); + haServices.setDispatcherLeaderElectionService(testingLeaderElectionService); + HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); + final JobID jobId = new JobID(); + + final TestingDispatcher dispatcher = new TestingDispatcher( + rpcService, + Dispatcher.DISPATCHER_NAME, + new Configuration(), + haServices, + mock(BlobServer.class), + heartbeatServices, + mock(MetricRegistry.class), + fatalErrorHandler, + mock(JobManagerRunner.class), + jobId); + + try { + dispatcher.start(); + + assertFalse(leaderSessionIdFuture.isDone()); + + testingLeaderElectionService.isLeader(expectedLeaderSessionId); + + UUID actualLeaderSessionId = leaderSessionIdFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + assertEquals(expectedLeaderSessionId, actualLeaderSessionId); + + verify(mockSubmittedJobGraphStore, Mockito.timeout(timeout.toMilliseconds()).atLeast(1)).getJobIds(); + } finally { + RpcUtils.terminateRpcEndpoint(dispatcher, timeout); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java index 0a7e9c848b19b..dba7bef5fa264 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -38,12 +38,16 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private volatile LeaderRetrievalService resourceManagerLeaderRetriever; + private volatile LeaderRetrievalService dispatcherLeaderRetriever; + private ConcurrentHashMap jobMasterLeaderRetrievers = new ConcurrentHashMap<>(); private ConcurrentHashMap jobManagerLeaderElectionServices = new ConcurrentHashMap<>(); private volatile LeaderElectionService resourceManagerLeaderElectionService; + private volatile LeaderElectionService dispatcherLeaderElectionService; + private volatile CheckpointRecoveryFactory checkpointRecoveryFactory; private volatile SubmittedJobGraphStore submittedJobGraphStore; @@ -56,6 +60,10 @@ public void setResourceManagerLeaderRetriever(LeaderRetrievalService resourceMan this.resourceManagerLeaderRetriever = resourceManagerLeaderRetriever; } + public void setDispatcherLeaderRetriever(LeaderRetrievalService dispatcherLeaderRetriever) { + this.dispatcherLeaderRetriever = dispatcherLeaderRetriever; + } + public void setJobMasterLeaderRetriever(JobID jobID, LeaderRetrievalService jobMasterLeaderRetriever) { this.jobMasterLeaderRetrievers.put(jobID, jobMasterLeaderRetriever); } @@ -68,6 +76,10 @@ public void setResourceManagerLeaderElectionService(LeaderElectionService leader this.resourceManagerLeaderElectionService = leaderElectionService; } + public void setDispatcherLeaderElectionService(LeaderElectionService leaderElectionService) { + this.dispatcherLeaderElectionService = leaderElectionService; + } + public void setCheckpointRecoveryFactory(CheckpointRecoveryFactory checkpointRecoveryFactory) { this.checkpointRecoveryFactory = checkpointRecoveryFactory; } @@ -90,6 +102,16 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() { } } + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + LeaderRetrievalService service = this.dispatcherLeaderRetriever; + if (service != null) { + return service; + } else { + throw new IllegalStateException("ResourceManagerLeaderRetriever has not been set"); + } + } + @Override public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { LeaderRetrievalService service = this.jobMasterLeaderRetrievers.get(jobID); @@ -116,6 +138,17 @@ public LeaderElectionService getResourceManagerLeaderElectionService() { } } + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + LeaderElectionService service = dispatcherLeaderElectionService; + + if (service != null) { + return service; + } else { + throw new IllegalStateException("DispatcherLeaderElectionService has not been set"); + } + } + @Override public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { LeaderElectionService service = this.jobManagerLeaderElectionServices.get(jobID); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java index 0735d17a31bf0..1f319ebfa6c61 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java @@ -45,9 +45,12 @@ public class TestingManualHighAvailabilityServices implements HighAvailabilitySe private final ManualLeaderService resourceManagerLeaderService; + private final ManualLeaderService dispatcherLeaderService; + public TestingManualHighAvailabilityServices() { jobManagerLeaderServices = new HashMap<>(4); resourceManagerLeaderService = new ManualLeaderService(); + dispatcherLeaderService = new ManualLeaderService(); } @Override @@ -55,6 +58,11 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() { return resourceManagerLeaderService.createLeaderRetrievalService(); } + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + return dispatcherLeaderService.createLeaderRetrievalService(); + } + @Override public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { ManualLeaderService leaderService = getOrCreateJobManagerLeaderService(jobID); @@ -72,6 +80,11 @@ public LeaderElectionService getResourceManagerLeaderElectionService() { return resourceManagerLeaderService.createLeaderElectionService(); } + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + return dispatcherLeaderService.createLeaderElectionService(); + } + @Override public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { ManualLeaderService leaderService = getOrCreateJobManagerLeaderService(jobID); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java index 2d51360777660..1cf2e5bb4ebdb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java @@ -39,6 +39,7 @@ public class StandaloneHaServicesTest extends TestLogger { private final String jobManagerAddress = "jobManager"; + private final String dispatcherAddress = "dispatcher"; private final String resourceManagerAddress = "resourceManager"; private StandaloneHaServices standaloneHaServices; @@ -46,7 +47,10 @@ public class StandaloneHaServicesTest extends TestLogger { @Before public void setupTest() { - standaloneHaServices = new StandaloneHaServices(resourceManagerAddress, jobManagerAddress); + standaloneHaServices = new StandaloneHaServices( + resourceManagerAddress, + dispatcherAddress, + jobManagerAddress); } @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 6842bee4a4057..e62213008bcbf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -462,6 +462,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { final ResourceID resourceID = ResourceID.generate(); final String resourceManagerAddress = "/resource/manager/address/one"; final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress); + final String dispatcherAddress = "localhost"; final String jobManagerAddress = "localhost"; final TestingRpcService rpc = new TestingRpcService(); @@ -483,6 +484,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { StandaloneHaServices haServices = new StandaloneHaServices( resourceManagerAddress, + dispatcherAddress, jobManagerAddress); final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java index 75f8c0a240384..86db1c42c4855 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnIntraNonHaMasterServices.java @@ -71,6 +71,9 @@ public class YarnIntraNonHaMasterServices extends AbstractYarnNonHaServices { /** The embedded leader election service used by JobManagers to find the resource manager. */ private final SingleLeaderElectionService resourceManagerLeaderElectionService; + /** The embedded leader election service for the dispatcher. */ + private final SingleLeaderElectionService dispatcherLeaderElectionService; + // ------------------------------------------------------------------------ /** @@ -100,6 +103,7 @@ public YarnIntraNonHaMasterServices( try { this.dispatcher = Executors.newSingleThreadExecutor(new ServicesThreadFactory()); this.resourceManagerLeaderElectionService = new SingleLeaderElectionService(dispatcher, DEFAULT_LEADER_ID); + this.dispatcherLeaderElectionService = new SingleLeaderElectionService(dispatcher, DEFAULT_LEADER_ID); // all good! successful = true; @@ -129,6 +133,17 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() { } } + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + enter(); + + try { + return dispatcherLeaderElectionService.createLeaderRetrievalService(); + } finally { + exit(); + } + } + @Override public LeaderElectionService getResourceManagerLeaderElectionService() { enter(); @@ -140,6 +155,16 @@ public LeaderElectionService getResourceManagerLeaderElectionService() { } } + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + enter(); + try { + return dispatcherLeaderElectionService; + } finally { + exit(); + } + } + @Override public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { enter(); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java index 6686a52fa0603..c1466d21da692 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/highavailability/YarnPreConfiguredMasterNonHaServices.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -66,6 +67,9 @@ public class YarnPreConfiguredMasterNonHaServices extends AbstractYarnNonHaServi /** The RPC URL under which the single ResourceManager can be reached while available. */ private final String resourceManagerRpcUrl; + /** The RPC URL under which the single Dispatcher can be reached while available. */ + private final String dispatcherRpcUrl; + // ------------------------------------------------------------------------ /** @@ -116,6 +120,13 @@ public YarnPreConfiguredMasterNonHaServices( addressResolution, config); + this.dispatcherRpcUrl = AkkaRpcServiceUtils.getRpcUrl( + rmHost, + rmPort, + Dispatcher.DISPATCHER_NAME, + addressResolution, + config); + // all well! successful = true; } @@ -144,6 +155,17 @@ public LeaderRetrievalService getResourceManagerLeaderRetriever() { } } + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + enter(); + + try { + return new StandaloneLeaderRetrievalService(dispatcherRpcUrl, DEFAULT_LEADER_ID); + } finally { + exit(); + } + } + @Override public LeaderElectionService getResourceManagerLeaderElectionService() { enter(); @@ -155,6 +177,16 @@ public LeaderElectionService getResourceManagerLeaderElectionService() { } } + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + enter(); + try { + throw new UnsupportedOperationException("Not supported on the TaskManager side"); + } finally { + exit(); + } + } + @Override public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { enter(); From 1fc4a609761445ecf77c374ae9daec8a1ada2618 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 22 Aug 2017 16:33:05 +0200 Subject: [PATCH 078/129] [FLINK-7489] Remove startJobExecution and suspendExecution from JobMasterGateway The job lifecycle methods should not be exposed as RPCs. Therefore, this commit removes them from the JobMasterGateway definition. This closes #4573. --- .../runtime/jobmaster/JobManagerRunner.java | 2 +- .../flink/runtime/jobmaster/JobMaster.java | 199 ++++++++++-------- .../runtime/jobmaster/JobMasterGateway.java | 10 - .../jobmaster/JobManagerRunnerMockTest.java | 4 +- 4 files changed, 109 insertions(+), 106 deletions(-) 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 c312cd361e932..8f1be4c7a1901 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 @@ -434,7 +434,7 @@ public void revokeLeadership() { log.info("JobManager for job {} ({}) was revoked leadership at {}.", jobGraph.getName(), jobGraph.getJobID(), getAddress()); - jobManager.getSelfGateway(JobMasterGateway.class).suspendExecution(new Exception("JobManager is no longer the leader.")); + jobManager.suspend(new Exception("JobManager is no longer the leader.")); } } 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 a8a8632ccaca7..c30749c9eacae 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 @@ -318,13 +318,28 @@ public void start(final UUID leaderSessionID) throws Exception { super.start(); log.info("JobManager started as leader {} for job {}", leaderSessionID, jobGraph.getJobID()); - selfGateway.startJobExecution(); + runAsync(this::startJobExecution); } else { log.warn("Job already started with leader ID {}, ignoring this start request.", leaderSessionID); } } + /** + * Suspending job, all the running tasks will be cancelled, and communication with other components + * will be disposed. + * + *

Mostly job is suspended because of the leadership has been revoked, one can be restart this job by + * calling the {@link #start(UUID)} method once we take the leadership back again. + * + *

This method is executed asynchronously + * + * @param cause The reason of why this job been suspended. + */ + public void suspend(final Throwable cause) { + runAsync(() -> suspendExecution(cause)); + } + /** * Suspend the job and shutdown all other services including rpc. */ @@ -343,98 +358,6 @@ public void postStop() throws Exception { // RPC methods //---------------------------------------------------------------------------------------------- - //-- job starting and stopping ----------------------------------------------------------------- - - @Override - public void startJobExecution() { - // double check that the leader status did not change - if (leaderSessionID == null) { - log.info("Aborting job startup - JobManager lost leader status"); - return; - } - - log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID()); - - try { - // start the slot pool make sure the slot pool now accepts messages for this leader - log.debug("Staring SlotPool component"); - slotPool.start(leaderSessionID, getAddress()); - } catch (Exception e) { - log.error("Faild to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), e); - - handleFatalError(new Exception("Could not start job execution: Failed to start the slot pool.", e)); - } - - try { - // job is ready to go, try to establish connection with resource manager - // - activate leader retrieval for the resource manager - // - on notification of the leader, the connection will be established and - // the slot pool will start requesting slots - resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener()); - } - catch (Throwable t) { - log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t); - - handleFatalError(new Exception( - "Could not start job execution: Failed to start leader service for Resource Manager", t)); - - return; - } - - // start scheduling job in another thread - executor.execute(new Runnable() { - @Override - public void run() { - try { - executionGraph.scheduleForExecution(); - } - catch (Throwable t) { - executionGraph.failGlobal(t); - } - } - }); - } - - /** - * Suspending job, all the running tasks will be cancelled, and communication with other components - * will be disposed. - * - *

Mostly job is suspended because of the leadership has been revoked, one can be restart this job by - * calling the {@link #start(UUID)} method once we take the leadership back again. - * - * @param cause The reason of why this job been suspended. - */ - @Override - public void suspendExecution(final Throwable cause) { - if (leaderSessionID == null) { - log.debug("Job has already been suspended or shutdown."); - return; - } - - // not leader any more - should not accept any leader messages any more - leaderSessionID = null; - - try { - resourceManagerLeaderRetriever.stop(); - } catch (Throwable t) { - log.warn("Failed to stop resource manager leader retriever when suspending.", t); - } - - // tell the execution graph (JobManager is still processing messages here) - executionGraph.suspend(cause); - - // receive no more messages until started again, should be called before we clear self leader id - stop(); - - // the slot pool stops receiving messages and clears its pooled slots - slotPoolGateway.suspend(); - - // disconnect from resource manager: - closeResourceManagerConnection(new Exception("Execution was suspended.", cause)); - } - - //---------------------------------------------------------------------------------------------- - /** * Updates the task execution state for a given task. * @@ -863,6 +786,96 @@ public void heartbeatFromResourceManager(final ResourceID resourceID) { // Internal methods //---------------------------------------------------------------------------------------------- + //-- job starting and stopping ----------------------------------------------------------------- + + private void startJobExecution() { + // double check that the leader status did not change + if (leaderSessionID == null) { + log.info("Aborting job startup - JobManager lost leader status"); + return; + } + + log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID()); + + try { + // start the slot pool make sure the slot pool now accepts messages for this leader + log.debug("Staring SlotPool component"); + slotPool.start(leaderSessionID, getAddress()); + } catch (Exception e) { + log.error("Faild to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), e); + + handleFatalError(new Exception("Could not start job execution: Failed to start the slot pool.", e)); + } + + try { + // job is ready to go, try to establish connection with resource manager + // - activate leader retrieval for the resource manager + // - on notification of the leader, the connection will be established and + // the slot pool will start requesting slots + resourceManagerLeaderRetriever.start(new ResourceManagerLeaderListener()); + } + catch (Throwable t) { + log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t); + + handleFatalError(new Exception( + "Could not start job execution: Failed to start leader service for Resource Manager", t)); + + return; + } + + // start scheduling job in another thread + executor.execute(new Runnable() { + @Override + public void run() { + try { + executionGraph.scheduleForExecution(); + } + catch (Throwable t) { + executionGraph.failGlobal(t); + } + } + }); + } + + /** + * Suspending job, all the running tasks will be cancelled, and communication with other components + * will be disposed. + * + *

Mostly job is suspended because of the leadership has been revoked, one can be restart this job by + * calling the {@link #start(UUID)} method once we take the leadership back again. + * + * @param cause The reason of why this job been suspended. + */ + private void suspendExecution(final Throwable cause) { + if (leaderSessionID == null) { + log.debug("Job has already been suspended or shutdown."); + return; + } + + // not leader any more - should not accept any leader messages any more + leaderSessionID = null; + + try { + resourceManagerLeaderRetriever.stop(); + } catch (Throwable t) { + log.warn("Failed to stop resource manager leader retriever when suspending.", t); + } + + // tell the execution graph (JobManager is still processing messages here) + executionGraph.suspend(cause); + + // receive no more messages until started again, should be called before we clear self leader id + stop(); + + // the slot pool stops receiving messages and clears its pooled slots + slotPoolGateway.suspend(); + + // disconnect from resource manager: + closeResourceManagerConnection(new Exception("Execution was suspended.", cause)); + } + + //---------------------------------------------------------------------------------------------- + private void handleFatalError(final Throwable cause) { runAsync(new Runnable() { @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 b396cd68df0b1..bfa29308c2ae6 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 @@ -52,16 +52,6 @@ */ public interface JobMasterGateway extends CheckpointCoordinatorGateway { - // ------------------------------------------------------------------------ - // Job start and stop methods - // ------------------------------------------------------------------------ - - void startJobExecution(); - - void suspendExecution(Throwable cause); - - // ------------------------------------------------------------------------ - /** * Updates the task execution state for a given task. * diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java index 435c23dcd0c61..998e803b83148 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java @@ -209,7 +209,7 @@ public void testLeadershipRevoked() throws Exception { assertTrue(!jobCompletion.isJobFinished()); runner.revokeLeadership(); - verify(jobManager).suspendExecution(any(Throwable.class)); + verify(jobManager).suspend(any(Throwable.class)); assertFalse(runner.isShutdown()); } @@ -224,7 +224,7 @@ public void testRegainLeadership() throws Exception { assertTrue(!jobCompletion.isJobFinished()); runner.revokeLeadership(); - verify(jobManager).suspendExecution(any(Throwable.class)); + verify(jobManager).suspend(any(Throwable.class)); assertFalse(runner.isShutdown()); UUID leaderSessionID2 = UUID.randomUUID(); From 0f3de89af4ef4f570e125b0d50110bfa5d0ce80b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 24 Aug 2017 15:29:47 +0200 Subject: [PATCH 079/129] [FLINK-7501] Generalize RegisteredRpcConnection to support generic leader ids The RegisteredRpcConnection now supports generic leader ids/fencing tokens. This will allow to introduce component specific leader ids/fencing tokens. This closes #4580. --- .../flink/runtime/jobmaster/JobMaster.java | 6 +-- .../registration/RegisteredRpcConnection.java | 37 ++++++++-------- .../registration/RetryingRegistration.java | 43 ++++++++++--------- .../taskexecutor/JobLeaderService.java | 8 ++-- ...skExecutorToResourceManagerConnection.java | 6 +-- .../RegisteredRpcConnectionTest.java | 4 +- .../RetryingRegistrationTest.java | 2 +- 7 files changed, 54 insertions(+), 52 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 c30749c9eacae..7e48da15f9e3e 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 @@ -1058,7 +1058,7 @@ public void handleError(final Exception exception) { //---------------------------------------------------------------------------------------------- private class ResourceManagerConnection - extends RegisteredRpcConnection + extends RegisteredRpcConnection { private final JobID jobID; @@ -1088,8 +1088,8 @@ private class ResourceManagerConnection } @Override - protected RetryingRegistration generateRegistration() { - return new RetryingRegistration( + protected RetryingRegistration generateRegistration() { + return new RetryingRegistration( log, getRpcService(), "ResourceManager", ResourceManagerGateway.class, getTargetAddress(), getTargetLeaderId()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java index da46e1c10d963..a585f0dff6881 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegisteredRpcConnection.java @@ -23,7 +23,7 @@ import org.slf4j.Logger; -import java.util.UUID; +import java.io.Serializable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -39,16 +39,17 @@ * The RPC connection can be closed, for example when the target where it tries to register * at looses leader status. * - * @param The type of the gateway to connect to. - * @param The type of the successful registration responses. + * @param The type of the fencing token + * @param The type of the gateway to connect to. + * @param The type of the successful registration responses. */ -public abstract class RegisteredRpcConnection { +public abstract class RegisteredRpcConnection { /** The logger for all log messages of this class. */ protected final Logger log; - /** The target component leaderID, for example the ResourceManager leaderID. */ - private final UUID targetLeaderId; + /** The fencing token fo the remote component. */ + private final F fencingToken; /** The target component Address, for example the ResourceManager Address. */ private final String targetAddress; @@ -57,20 +58,20 @@ public abstract class RegisteredRpcConnection pendingRegistration; + private RetryingRegistration pendingRegistration; /** The gateway to register, it's null until the registration is completed. */ - private volatile Gateway targetGateway; + private volatile G targetGateway; /** Flag indicating that the RPC connection is closed. */ private volatile boolean closed; // ------------------------------------------------------------------------ - public RegisteredRpcConnection(Logger log, String targetAddress, UUID targetLeaderId, Executor executor) { + public RegisteredRpcConnection(Logger log, String targetAddress, F fencingToken, Executor executor) { this.log = checkNotNull(log); this.targetAddress = checkNotNull(targetAddress); - this.targetLeaderId = checkNotNull(targetLeaderId); + this.fencingToken = checkNotNull(fencingToken); this.executor = checkNotNull(executor); } @@ -86,10 +87,10 @@ public void start() { pendingRegistration = checkNotNull(generateRegistration()); pendingRegistration.startRegistration(); - CompletableFuture> future = pendingRegistration.getFuture(); + CompletableFuture> future = pendingRegistration.getFuture(); future.whenCompleteAsync( - (Tuple2 result, Throwable failure) -> { + (Tuple2 result, Throwable failure) -> { // this future should only ever fail if there is a bug, not if the registration is declined if (failure != null) { onRegistrationFailure(failure); @@ -103,12 +104,12 @@ public void start() { /** * This method generate a specific Registration, for example TaskExecutor Registration at the ResourceManager. */ - protected abstract RetryingRegistration generateRegistration(); + protected abstract RetryingRegistration generateRegistration(); /** * This method handle the Registration Response. */ - protected abstract void onRegistrationSuccess(Success success); + protected abstract void onRegistrationSuccess(S success); /** * This method handle the Registration failure. @@ -135,8 +136,8 @@ public boolean isClosed() { // Properties // ------------------------------------------------------------------------ - public UUID getTargetLeaderId() { - return targetLeaderId; + public F getTargetLeaderId() { + return fencingToken; } public String getTargetAddress() { @@ -146,7 +147,7 @@ public String getTargetAddress() { /** * Gets the RegisteredGateway. This returns null until the registration is completed. */ - public Gateway getTargetGateway() { + public G getTargetGateway() { return targetGateway; } @@ -158,7 +159,7 @@ public boolean isConnected() { @Override public String toString() { - String connectionInfo = "(ADDRESS: " + targetAddress + " LEADERID: " + targetLeaderId + ")"; + String connectionInfo = "(ADDRESS: " + targetAddress + " FENCINGTOKEN: " + fencingToken + ")"; if (isConnected()) { connectionInfo = "RPC connection to " + targetGateway.getClass().getSimpleName() + " " + connectionInfo; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java index 6a18ffd78b5ed..be30c6838dbe5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java @@ -24,7 +24,7 @@ import org.slf4j.Logger; -import java.util.UUID; +import java.io.Serializable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -43,10 +43,11 @@ * The registration can be canceled, for example when the target where it tries to register * at looses leader status. * - * @param The type of the gateway to connect to. - * @param The type of the successful registration responses. + * @param The type of the fencing token + * @param The type of the gateway to connect to. + * @param The type of the successful registration responses. */ -public abstract class RetryingRegistration { +public abstract class RetryingRegistration { // ------------------------------------------------------------------------ // default configuration values @@ -74,13 +75,13 @@ public abstract class RetryingRegistration targetType; + private final Class targetType; private final String targetAddress; - private final UUID leaderId; + private final F fencingToken; - private final CompletableFuture> completionFuture; + private final CompletableFuture> completionFuture; private final long initialRegistrationTimeout; @@ -98,10 +99,10 @@ public RetryingRegistration( Logger log, RpcService rpcService, String targetName, - Class targetType, + Class targetType, String targetAddress, - UUID leaderId) { - this(log, rpcService, targetName, targetType, targetAddress, leaderId, + F fencingToken) { + this(log, rpcService, targetName, targetType, targetAddress, fencingToken, INITIAL_REGISTRATION_TIMEOUT_MILLIS, MAX_REGISTRATION_TIMEOUT_MILLIS, ERROR_REGISTRATION_DELAY_MILLIS, REFUSED_REGISTRATION_DELAY_MILLIS); } @@ -110,9 +111,9 @@ public RetryingRegistration( Logger log, RpcService rpcService, String targetName, - Class targetType, + Class targetType, String targetAddress, - UUID leaderId, + F fencingToken, long initialRegistrationTimeout, long maxRegistrationTimeout, long delayOnError, @@ -128,7 +129,7 @@ public RetryingRegistration( this.targetName = checkNotNull(targetName); this.targetType = checkNotNull(targetType); this.targetAddress = checkNotNull(targetAddress); - this.leaderId = checkNotNull(leaderId); + this.fencingToken = checkNotNull(fencingToken); this.initialRegistrationTimeout = initialRegistrationTimeout; this.maxRegistrationTimeout = maxRegistrationTimeout; this.delayOnError = delayOnError; @@ -141,7 +142,7 @@ public RetryingRegistration( // completion and cancellation // ------------------------------------------------------------------------ - public CompletableFuture> getFuture() { + public CompletableFuture> getFuture() { return completionFuture; } @@ -165,7 +166,7 @@ public boolean isCanceled() { // ------------------------------------------------------------------------ protected abstract CompletableFuture invokeRegistration( - Gateway gateway, UUID leaderId, long timeoutMillis) throws Exception; + G gateway, F fencingToken, long timeoutMillis) throws Exception; /** * This method resolves the target address to a callable gateway and starts the @@ -175,11 +176,11 @@ protected abstract CompletableFuture invokeRegistration( public void startRegistration() { try { // trigger resolution of the resource manager address to a callable gateway - CompletableFuture resourceManagerFuture = rpcService.connect(targetAddress, targetType); + CompletableFuture resourceManagerFuture = rpcService.connect(targetAddress, targetType); // upon success, start the registration attempts CompletableFuture resourceManagerAcceptFuture = resourceManagerFuture.thenAcceptAsync( - (Gateway result) -> { + (G result) -> { log.info("Resolved {} address, beginning registration", targetName); register(result, 1, initialRegistrationTimeout); }, @@ -206,7 +207,7 @@ public void startRegistration() { * depending on the result. */ @SuppressWarnings("unchecked") - private void register(final Gateway gateway, final int attempt, final long timeoutMillis) { + private void register(final G gateway, final int attempt, final long timeoutMillis) { // eager check for canceling to avoid some unnecessary work if (canceled) { return; @@ -214,7 +215,7 @@ private void register(final Gateway gateway, final int attempt, final long timeo try { log.info("Registration at {} attempt {} (timeout={}ms)", targetName, attempt, timeoutMillis); - CompletableFuture registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis); + CompletableFuture registrationFuture = invokeRegistration(gateway, fencingToken, timeoutMillis); // if the registration was successful, let the TaskExecutor know CompletableFuture registrationAcceptFuture = registrationFuture.thenAcceptAsync( @@ -222,7 +223,7 @@ private void register(final Gateway gateway, final int attempt, final long timeo if (!isCanceled()) { if (result instanceof RegistrationResponse.Success) { // registration successful! - Success success = (Success) result; + S success = (S) result; completionFuture.complete(Tuple2.of(gateway, success)); } else { @@ -274,7 +275,7 @@ private void register(final Gateway gateway, final int attempt, final long timeo } } - private void registerLater(final Gateway gateway, final int attempt, final long timeoutMillis, long delay) { + private void registerLater(final G gateway, final int attempt, final long timeoutMillis, long delay) { rpcService.scheduleRunnable(new Runnable() { @Override public void run() { 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 2ebf3c1f2dc24..6d1f22c20acc9 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 @@ -210,7 +210,7 @@ private final class JobManagerLeaderListener implements LeaderRetrievalListener private final JobID jobId; /** Rpc connection to the job leader */ - private RegisteredRpcConnection rpcConnection; + private RegisteredRpcConnection rpcConnection; /** State of the listener */ private volatile boolean stopped; @@ -299,7 +299,7 @@ public void handleError(Exception exception) { /** * Rpc connection for the job manager <--> task manager connection. */ - private final class JobManagerRegisteredRpcConnection extends RegisteredRpcConnection { + private final class JobManagerRegisteredRpcConnection extends RegisteredRpcConnection { JobManagerRegisteredRpcConnection( Logger log, @@ -310,7 +310,7 @@ private final class JobManagerRegisteredRpcConnection extends RegisteredRpcConne } @Override - protected RetryingRegistration generateRegistration() { + protected RetryingRegistration generateRegistration() { return new JobLeaderService.JobManagerRetryingRegistration( LOG, rpcService, @@ -351,7 +351,7 @@ protected void onRegistrationFailure(Throwable failure) { * Retrying registration for the job manager <--> task manager connection. */ private static final class JobManagerRetryingRegistration - extends RetryingRegistration + extends RetryingRegistration { private final String taskManagerRpcAddress; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java index 4084d67a60055..24eb540b76dda 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -41,7 +41,7 @@ * The connection between a TaskExecutor and the ResourceManager. */ public class TaskExecutorToResourceManagerConnection - extends RegisteredRpcConnection { + extends RegisteredRpcConnection { private final RpcService rpcService; @@ -79,7 +79,7 @@ public TaskExecutorToResourceManagerConnection( @Override - protected RetryingRegistration generateRegistration() { + protected RetryingRegistration generateRegistration() { return new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration( log, rpcService, @@ -127,7 +127,7 @@ public ResourceID getResourceManagerId() { // ------------------------------------------------------------------------ private static class ResourceManagerRegistration - extends RetryingRegistration { + extends RetryingRegistration { private final String taskExecutorAddress; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java index a4548673531fc..19a57563c3815 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RegisteredRpcConnectionTest.java @@ -141,7 +141,7 @@ public void testRpcConnectionClose() throws Exception { // test RegisteredRpcConnection // ------------------------------------------------------------------------ - private static class TestRpcConnection extends RegisteredRpcConnection { + private static class TestRpcConnection extends RegisteredRpcConnection { private final RpcService rpcService; @@ -155,7 +155,7 @@ public TestRpcConnection(String targetAddress, UUID targetLeaderId, Executor exe } @Override - protected RetryingRegistration generateRegistration() { + protected RetryingRegistration generateRegistration() { return new RetryingRegistrationTest.TestRetryingRegistration(rpcService, getTargetAddress(), getTargetLeaderId()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java index da992bb103b57..ac0dbc57c848a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java @@ -320,7 +320,7 @@ public String getCorrelationId() { } } - static class TestRetryingRegistration extends RetryingRegistration { + static class TestRetryingRegistration extends RetryingRegistration { // we use shorter timeouts here to speed up the tests static final long INITIAL_TIMEOUT = 20; From 0cf7f7666a00dbe0d19d5078ef9f14cdb00ed17c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 14 Aug 2017 15:47:04 +0200 Subject: [PATCH 080/129] [FLINK-7522] Add termination future to ClusterEntrypoint The termination future is completed when the ClusterEntrypoint shuts down. This allows for easier testing. This closes #4589. --- .../flink/runtime/entrypoint/ClusterEntrypoint.java | 10 ++++++++++ 1 file changed, 10 insertions(+) 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 1551933b4b116..861355fd3490b 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 @@ -49,6 +49,7 @@ import java.io.IOException; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import scala.concurrent.duration.FiniteDuration; @@ -71,6 +72,8 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler { private final Configuration configuration; + private final CompletableFuture terminationFuture; + @GuardedBy("lock") private MetricRegistry metricRegistry = null; @@ -88,6 +91,11 @@ public abstract class ClusterEntrypoint implements FatalErrorHandler { protected ClusterEntrypoint(Configuration configuration) { this.configuration = Preconditions.checkNotNull(configuration); + this.terminationFuture = new CompletableFuture<>(); + } + + public CompletableFuture getTerminationFuture() { + return terminationFuture; } protected void startCluster() { @@ -246,6 +254,8 @@ protected void shutDown(boolean cleanupHaData) throws FlinkException { exception = ExceptionUtils.firstOrSuppressed(t, exception); } } + + terminationFuture.complete(true); } if (exception != null) { From d7cea586ec60f89ed06baf7ab95ebcf54f42a537 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Fri, 25 Aug 2017 12:15:38 +0200 Subject: [PATCH 081/129] [FLINK-7519] Add HttpResponseStatus to RestClientException Enrich the RestClientException with the HttpResponseStatus in case of failures. This makes the exception handling on the client side easier. This closes #4588. --- .../apache/flink/runtime/rest/RestClient.java | 76 ++++++++++++++----- .../rest/handler/AbstractRestHandler.java | 2 +- .../rest/handler/RestHandlerException.java | 10 +-- .../rest/util/RestClientException.java | 20 +++-- 4 files changed, 78 insertions(+), 30 deletions(-) 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 7422ecefd633f..ea266be789508 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 @@ -48,6 +48,8 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpClientCodec; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaders; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpObjectAggregator; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse; +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpVersion; import org.apache.flink.shaded.netty4.io.netty.handler.ssl.SslHandler; @@ -85,15 +87,15 @@ public RestClient(RestClientConfiguration configuration, Executor executor) { this.executor = Preconditions.checkNotNull(executor); SSLEngine sslEngine = configuration.getSslEngine(); - ChannelInitializer initializer = new ChannelInitializer() { + ChannelInitializer initializer = new ChannelInitializer() { @Override - protected void initChannel(SocketChannel ch) throws Exception { + protected void initChannel(SocketChannel socketChannel) throws Exception { // SSL should be the first handler in the pipeline if (sslEngine != null) { - ch.pipeline().addLast("ssl", new SslHandler(sslEngine)); + socketChannel.pipeline().addLast("ssl", new SslHandler(sslEngine)); } - ch.pipeline() + socketChannel.pipeline() .addLast(new HttpClientCodec()) .addLast(new HttpObjectAggregator(1024 * 1024)) .addLast(new ClientHandler()) @@ -150,7 +152,7 @@ public , U extends MessageParameters, R extend httpRequest.headers() .add(HttpHeaders.Names.CONTENT_LENGTH, payload.capacity()) .add(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ConfigConstants.DEFAULT_CHARSET.name()) - .set(HttpHeaders.Names.HOST, targetAddress + ":" + targetPort) + .set(HttpHeaders.Names.HOST, targetAddress + ':' + targetPort) .set(HttpHeaders.Names.CONNECTION, HttpHeaders.Values.CLOSE); return submitRequest(targetAddress, targetPort, httpRequest, messageHeaders.getResponseClass()); @@ -168,29 +170,36 @@ private

CompletableFuture

submitRequest(String targe .thenApply((ChannelFuture::channel)) .thenCompose(channel -> { ClientHandler handler = channel.pipeline().get(ClientHandler.class); - CompletableFuture future = handler.getJsonFuture(); + CompletableFuture future = handler.getJsonFuture(); channel.writeAndFlush(httpRequest); - return future.thenComposeAsync(rawResponse -> parseResponse(rawResponse, responseClass), executor); - }); + return future; + }).thenComposeAsync( + (JsonResponse rawResponse) -> parseResponse(rawResponse, responseClass), + executor + ); } - private static

CompletableFuture

parseResponse(JsonNode rawResponse, Class

responseClass) { + private static

CompletableFuture

parseResponse(JsonResponse rawResponse, Class

responseClass) { CompletableFuture

responseFuture = new CompletableFuture<>(); try { - P response = objectMapper.treeToValue(rawResponse, responseClass); + P response = objectMapper.treeToValue(rawResponse.getJson(), responseClass); responseFuture.complete(response); } catch (JsonProcessingException jpe) { // the received response did not matched the expected response type // lets see if it is an ErrorResponse instead try { - ErrorResponseBody error = objectMapper.treeToValue(rawResponse, ErrorResponseBody.class); - responseFuture.completeExceptionally(new RestClientException(error.errors.toString())); + ErrorResponseBody error = objectMapper.treeToValue(rawResponse.getJson(), ErrorResponseBody.class); + responseFuture.completeExceptionally(new RestClientException(error.errors.toString(), rawResponse.getHttpResponseStatus())); } catch (JsonProcessingException jpe2) { // if this fails it is either the expected type or response type was wrong, most likely caused // by a client/search MessageHeaders mismatch LOG.error("Received response was neither of the expected type ({}) nor an error. Response={}", responseClass, rawResponse, jpe2); - responseFuture.completeExceptionally(new RestClientException("Response was neither of the expected type(" + responseClass + ") nor an error.", jpe2)); + responseFuture.completeExceptionally( + new RestClientException( + "Response was neither of the expected type(" + responseClass + ") nor an error.", + jpe2, + rawResponse.getHttpResponseStatus())); } } return responseFuture; @@ -198,9 +207,9 @@ private static

CompletableFuture

parseResponse(JsonN private static class ClientHandler extends SimpleChannelInboundHandler { - private final CompletableFuture jsonFuture = new CompletableFuture<>(); + private final CompletableFuture jsonFuture = new CompletableFuture<>(); - CompletableFuture getJsonFuture() { + CompletableFuture getJsonFuture() { return jsonFuture; } @@ -210,7 +219,18 @@ protected void channelRead0(ChannelHandlerContext ctx, Object msg) { readRawResponse((FullHttpResponse) msg); } else { LOG.error("Implementation error: Received a response that wasn't a FullHttpResponse."); - jsonFuture.completeExceptionally(new RestClientException("Implementation error: Received a response that wasn't a FullHttpResponse.")); + if (msg instanceof HttpResponse) { + jsonFuture.completeExceptionally( + new RestClientException( + "Implementation error: Received a response that wasn't a FullHttpResponse.", + ((HttpResponse) msg).getStatus())); + } else { + jsonFuture.completeExceptionally( + new RestClientException( + "Implementation error: Received a response that wasn't a FullHttpResponse.", + HttpResponseStatus.INTERNAL_SERVER_ERROR)); + } + } ctx.close(); } @@ -225,14 +245,32 @@ private void readRawResponse(FullHttpResponse msg) { LOG.debug("Received response {}.", rawResponse); } catch (JsonParseException je) { LOG.error("Response was not valid JSON.", je); - jsonFuture.completeExceptionally(new RestClientException("Response was not valid JSON.", je)); + jsonFuture.completeExceptionally(new RestClientException("Response was not valid JSON.", je, msg.getStatus())); return; } catch (IOException ioe) { LOG.error("Response could not be read.", ioe); - jsonFuture.completeExceptionally(new RestClientException("Response could not be read.", ioe)); + jsonFuture.completeExceptionally(new RestClientException("Response could not be read.", ioe, msg.getStatus())); return; } - jsonFuture.complete(rawResponse); + jsonFuture.complete(new JsonResponse(rawResponse, msg.getStatus())); + } + } + + private static final class JsonResponse { + private final JsonNode json; + private final HttpResponseStatus httpResponseStatus; + + private JsonResponse(JsonNode json, HttpResponseStatus httpResponseStatus) { + this.json = Preconditions.checkNotNull(json); + this.httpResponseStatus = Preconditions.checkNotNull(httpResponseStatus); + } + + public JsonNode getJson() { + return json; + } + + public HttpResponseStatus getHttpResponseStatus() { + return httpResponseStatus; } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java index 23e29184e2df0..2f2f9aa034e16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractRestHandler.java @@ -135,7 +135,7 @@ protected void channelRead0(final ChannelHandlerContext ctx, Routed routed) thro if (error != null) { if (error instanceof RestHandlerException) { RestHandlerException rhe = (RestHandlerException) error; - sendErrorResponse(new ErrorResponseBody(rhe.getErrorMessage()), rhe.getErrorCode(), ctx, httpRequest); + sendErrorResponse(new ErrorResponseBody(rhe.getErrorMessage()), rhe.getHttpResponseStatus(), ctx, httpRequest); } else { log.error("Implementation error: Unhandled exception.", error); sendErrorResponse(new ErrorResponseBody("Internal server error."), HttpResponseStatus.INTERNAL_SERVER_ERROR, ctx, httpRequest); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerException.java index 9285f25bbd116..4cbb542ada7d7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerException.java @@ -27,18 +27,18 @@ public class RestHandlerException extends Exception { private static final long serialVersionUID = -1358206297964070876L; private final String errorMessage; - private final HttpResponseStatus errorCode; + private final int responseCode; - public RestHandlerException(String errorMessage, HttpResponseStatus errorCode) { + public RestHandlerException(String errorMessage, HttpResponseStatus httpResponseStatus) { this.errorMessage = errorMessage; - this.errorCode = errorCode; + this.responseCode = httpResponseStatus.code(); } public String getErrorMessage() { return errorMessage; } - public HttpResponseStatus getErrorCode() { - return errorCode; + public HttpResponseStatus getHttpResponseStatus() { + return HttpResponseStatus.valueOf(responseCode); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/util/RestClientException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/util/RestClientException.java index 9d86b479df0a9..2333614bfc16f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/util/RestClientException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/util/RestClientException.java @@ -19,6 +19,9 @@ package org.apache.flink.runtime.rest.util; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; /** * An exception that is thrown if the failure of a REST operation was detected on the client. @@ -27,15 +30,22 @@ public class RestClientException extends FlinkException { private static final long serialVersionUID = 937914622022344423L; - public RestClientException(String message) { + private final int responseCode; + + public RestClientException(String message, HttpResponseStatus responseStatus) { super(message); - } - public RestClientException(Throwable cause) { - super(cause); + Preconditions.checkNotNull(responseStatus); + responseCode = responseStatus.code(); } - public RestClientException(String message, Throwable cause) { + public RestClientException(String message, Throwable cause, HttpResponseStatus responseStatus) { super(message, cause); + + responseCode = responseStatus.code(); + } + + public HttpResponseStatus getHttpResponseStatus() { + return HttpResponseStatus.valueOf(responseCode); } } From 1804aa33d0996810ad35bfa4dd526c86a1e12828 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 23 Aug 2017 17:08:55 +0200 Subject: [PATCH 082/129] [FLINK-7078] [rpc] Introduce FencedRpcEndpoint Introduce FencedRpcEndpoint which requires all RPC messages to have a fencing token attached. Based on the received fencing token and the actual fencing token, the message will either be discarded if they are not equal or it will be processed. That way we are able to filter out old messages or messages which originate from a split brain situation Add support for callAsyncWithoutFencing Introduce local and remote fenced messages This closes #4578. --- .../rpc/FencedMainThreadExecutable.java | 48 +++ .../flink/runtime/rpc/FencedRpcEndpoint.java | 116 ++++++ .../flink/runtime/rpc/FencedRpcGateway.java | 37 ++ .../apache/flink/runtime/rpc/RpcEndpoint.java | 15 +- .../apache/flink/runtime/rpc/RpcService.java | 38 +- .../rpc/akka/AkkaInvocationHandler.java | 178 ++++++--- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 123 ++++--- .../runtime/rpc/akka/AkkaRpcService.java | 234 ++++++++---- .../rpc/akka/FencedAkkaInvocationHandler.java | 145 ++++++++ .../runtime/rpc/akka/FencedAkkaRpcActor.java | 76 ++++ .../AkkaUnknownMessageException.java | 40 ++ .../runtime/rpc/akka/messages/Processing.java | 2 +- .../FencingTokenMismatchException.java | 42 +++ .../rpc/{akka => }/messages/CallAsync.java | 2 +- .../runtime/rpc/messages/ControlMessage.java | 26 ++ .../runtime/rpc/messages/FencedMessage.java | 34 ++ .../rpc/messages/LocalFencedMessage.java | 56 +++ .../messages/LocalRpcInvocation.java | 27 +- .../rpc/messages/RemoteFencedMessage.java | 57 +++ .../messages/RemoteRpcInvocation.java | 33 +- .../{akka => }/messages/RpcInvocation.java | 2 +- .../rpc/{akka => }/messages/RunAsync.java | 4 +- .../rpc/{akka => }/messages/Shutdown.java | 4 +- .../runtime/rpc/messages/UnfencedMessage.java | 49 +++ .../flink/runtime/rpc/AsyncCallsTest.java | 157 ++++++++ .../runtime/rpc/FencedRpcEndpointTest.java | 344 ++++++++++++++++++ .../flink/runtime/rpc/TestingRpcService.java | 23 +- 27 files changed, 1724 insertions(+), 188 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedMainThreadExecutable.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcGateway.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/FencingTokenMismatchException.java rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/{akka => }/messages/CallAsync.java (96%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/ControlMessage.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/FencedMessage.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/LocalFencedMessage.java rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/{akka => }/messages/LocalRpcInvocation.java (72%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteFencedMessage.java rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/{akka => }/messages/RemoteRpcInvocation.java (88%) rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/{akka => }/messages/RpcInvocation.java (97%) rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/{akka => }/messages/RunAsync.java (95%) rename flink-runtime/src/main/java/org/apache/flink/runtime/rpc/{akka => }/messages/Shutdown.java (91%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/UnfencedMessage.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedMainThreadExecutable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedMainThreadExecutable.java new file mode 100644 index 0000000000000..16cacc84dd237 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedMainThreadExecutable.java @@ -0,0 +1,48 @@ +/* + * 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.rpc; + +import org.apache.flink.api.common.time.Time; + +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; + +/** + * Extended {@link MainThreadExecutable} interface which allows to run unfenced runnables + * in the main thread. + */ +public interface FencedMainThreadExecutable extends MainThreadExecutable { + + /** + * Run the given runnable in the main thread without attaching a fencing token. + * + * @param runnable to run in the main thread without validating the fencing token. + */ + void runAsyncWithoutFencing(Runnable runnable); + + /** + * Run the given callable in the main thread without attaching a fencing token. + * + * @param callable to run in the main thread without validating the fencing token. + * @param timeout for the operation + * @param type of the callable result + * @return Future containing the callable result + */ + CompletableFuture callAsyncWithoutFencing(Callable callable, Time timeout); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java new file mode 100644 index 0000000000000..81bae2924e674 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcEndpoint.java @@ -0,0 +1,116 @@ +/* + * 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.rpc; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; + +/** + * Base class for fenced {@link RpcEndpoint}. A fenced rpc endpoint expects all rpc messages + * being enriched with fencing tokens. Furthermore, the rpc endpoint has its own fencing token + * assigned. The rpc is then only executed if the attached fencing token equals the endpoint's own + * token. + * + * @param type of the fencing token + */ +public class FencedRpcEndpoint extends RpcEndpoint { + + private volatile F fencingToken; + private volatile MainThreadExecutor fencedMainThreadExecutor; + + protected FencedRpcEndpoint(RpcService rpcService, String endpointId, F initialFencingToken) { + super(rpcService, endpointId); + + this.fencingToken = Preconditions.checkNotNull(initialFencingToken); + this.fencedMainThreadExecutor = new MainThreadExecutor( + getRpcService().fenceRpcServer( + rpcServer, + initialFencingToken)); + } + + protected FencedRpcEndpoint(RpcService rpcService, F initialFencingToken) { + this(rpcService, UUID.randomUUID().toString(), initialFencingToken); + } + + public F getFencingToken() { + return fencingToken; + } + + protected void setFencingToken(F newFencingToken) { + // this method should only be called from within the main thread + validateRunsInMainThread(); + + this.fencingToken = newFencingToken; + + // setting a new fencing token entails that we need a new MainThreadExecutor + // which is bound to the new fencing token + MainThreadExecutable mainThreadExecutable = getRpcService().fenceRpcServer( + rpcServer, + newFencingToken); + + this.fencedMainThreadExecutor = new MainThreadExecutor(mainThreadExecutable); + } + + /** + * Returns a main thread executor which is bound to the currently valid fencing token. + * This means that runnables which are executed with this executor fail after the fencing + * token has changed. This allows to scope operations by the fencing token. + * + * @return MainThreadExecutor bound to the current fencing token + */ + @Override + protected MainThreadExecutor getMainThreadExecutor() { + return fencedMainThreadExecutor; + } + + /** + * Run the given runnable in the main thread of the RpcEndpoint without checking the fencing + * token. This allows to run operations outside of the fencing token scope. + * + * @param runnable to execute in the main thread of the rpc endpoint without checking the fencing token. + */ + protected void runAsyncWithoutFencing(Runnable runnable) { + if (rpcServer instanceof FencedMainThreadExecutable) { + ((FencedMainThreadExecutable) rpcServer).runAsyncWithoutFencing(runnable); + } else { + throw new RuntimeException("FencedRpcEndpoint has not been started with a FencedMainThreadExecutable RpcServer."); + } + } + + /** + * Run the given callable in the main thread of the RpcEndpoint without checking the fencing + * token. This allows to run operations outside of the fencing token scope. + * + * @param callable to run in the main thread of the rpc endpoint without checkint the fencing token. + * @param timeout for the operation. + * @return Future containing the callable result. + */ + protected CompletableFuture callAsyncWithoutFencing(Callable callable, Time timeout) { + if (rpcServer instanceof FencedMainThreadExecutable) { + return ((FencedMainThreadExecutable) rpcServer).callAsyncWithoutFencing(callable, timeout); + } else { + throw new RuntimeException("FencedRpcEndpoint has not been started with a FencedMainThreadExecutable RpcServer."); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcGateway.java new file mode 100644 index 0000000000000..fab638f912c1b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FencedRpcGateway.java @@ -0,0 +1,37 @@ +/* + * 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.rpc; + +import java.io.Serializable; + +/** + * Fenced {@link RpcGateway}. This gateway allows to have access to the associated + * fencing token. + * + * @param type of the fencing token + */ +public interface FencedRpcGateway extends RpcGateway { + + /** + * Get the current fencing token. + * + * @return current fencing token + */ + F getFencingToken(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java index 980ae48dfe991..563674add8685 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java @@ -24,6 +24,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; + import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -66,14 +67,14 @@ public abstract class RpcEndpoint implements RpcGateway { private final String endpointId; /** Interface to access the underlying rpc server */ - private final RpcServer rpcServer; + protected final RpcServer rpcServer; + + /** A reference to the endpoint's main thread, if the current method is called by the main thread */ + final AtomicReference currentMainThread = new AtomicReference<>(null); /** The main thread executor to be used to execute future callbacks in the main thread * of the executing rpc server. */ - private final Executor mainThreadExecutor; - - /** A reference to the endpoint's main thread, if the current method is called by the main thread */ - final AtomicReference currentMainThread = new AtomicReference<>(null); + private final MainThreadExecutor mainThreadExecutor; /** * Initializes the RPC endpoint. @@ -208,7 +209,7 @@ public String getHostname() { * * @return Main thread execution context */ - protected Executor getMainThreadExecutor() { + protected MainThreadExecutor getMainThreadExecutor() { return mainThreadExecutor; } @@ -310,7 +311,7 @@ public void validateRunsInMainThread() { /** * Executor which executes runnables in the main thread context. */ - private static class MainThreadExecutor implements Executor { + protected static class MainThreadExecutor implements Executor { private final MainThreadExecutable gateway; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java index 3b5a5e2c886f8..9b2e318888e63 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; +import java.io.Serializable; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -61,7 +62,27 @@ public interface RpcService { * @return Future containing the rpc gateway or an {@link RpcConnectionException} if the * connection attempt failed */ - CompletableFuture connect(String address, Class clazz); + CompletableFuture connect( + String address, + Class clazz); + + /** + * Connect to ta remote fenced rpc server under the provided address. Returns a fenced rpc gateway + * which can be used to communicate with the rpc server. If the connection failed, then the + * returned future is failed with a {@link RpcConnectionException}. + * + * @param address Address of the remote rpc server + * @param fencingToken Fencing token to be used when communicating with the server + * @param clazz Class of the rpc gateway to return + * @param Type of the fencing token + * @param Type of the rpc gateway to return + * @return Future containing the fenced rpc gateway or an {@link RpcConnectionException} if the + * connection attempt failed + */ + > CompletableFuture connect( + String address, + F fencingToken, + Class clazz); /** * Start a rpc server which forwards the remote procedure calls to the provided rpc endpoint. @@ -72,6 +93,21 @@ public interface RpcService { */ RpcServer startServer(C rpcEndpoint); + + /** + * Fence the given RpcServer with the given fencing token. + * + *

Fencing the RpcServer means that we fix the fencing token to the provided value. + * All RPCs will then be enriched with this fencing token. This expects that the receiving + * RPC endpoint extends {@link FencedRpcEndpoint}. + * + * @param rpcServer to fence with the given fencing token + * @param fencingToken to fence the RpcServer with + * @param type of the fencing token + * @return Fenced RpcServer + */ + RpcServer fenceRpcServer(RpcServer rpcServer, F fencingToken); + /** * Stop the underlying rpc server of the provided self gateway. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java index 0521f2e5717ea..fc785cb7c8d33 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java @@ -22,17 +22,18 @@ import akka.pattern.Patterns; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.MainThreadExecutable; import org.apache.flink.runtime.rpc.RpcServer; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.rpc.StartStoppable; -import org.apache.flink.runtime.rpc.akka.messages.CallAsync; -import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation; +import org.apache.flink.runtime.rpc.messages.CallAsync; +import org.apache.flink.runtime.rpc.messages.LocalRpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.Processing; -import org.apache.flink.runtime.rpc.akka.messages.RemoteRpcInvocation; -import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; -import org.apache.flink.runtime.rpc.akka.messages.RunAsync; +import org.apache.flink.runtime.rpc.messages.RemoteRpcInvocation; +import org.apache.flink.runtime.rpc.messages.RpcInvocation; +import org.apache.flink.runtime.rpc.messages.RunAsync; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,6 +47,7 @@ import java.util.Objects; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkArgument; @@ -72,7 +74,7 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, RpcServer private final ActorRef rpcEndpoint; // whether the actor ref is local and thus no message serialization is needed - private final boolean isLocal; + protected final boolean isLocal; // default timeout for asks private final Time timeout; @@ -112,53 +114,13 @@ public Object invoke(Object proxy, Method method, Object[] args) throws Throwabl declaringClass.equals(MainThreadExecutable.class) || declaringClass.equals(RpcServer.class)) { result = method.invoke(this, args); + } else if (declaringClass.equals(FencedRpcGateway.class)) { + throw new UnsupportedOperationException("AkkaInvocationHandler does not support the call FencedRpcGateway#" + + method.getName() + ". This indicates that you retrieved a FencedRpcGateway without specifying a " + + "fencing token. Please use RpcService#connect(RpcService, F, Time) with F being the fencing token to " + + "retrieve a properly FencedRpcGateway."); } else { - String methodName = method.getName(); - Class[] parameterTypes = method.getParameterTypes(); - Annotation[][] parameterAnnotations = method.getParameterAnnotations(); - Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); - - RpcInvocation rpcInvocation; - - if (isLocal) { - rpcInvocation = new LocalRpcInvocation( - methodName, - parameterTypes, - args); - } else { - try { - RemoteRpcInvocation remoteRpcInvocation = new RemoteRpcInvocation( - methodName, - parameterTypes, - args); - - if (remoteRpcInvocation.getSize() > maximumFramesize) { - throw new IOException("The rpc invocation size exceeds the maximum akka framesize."); - } else { - rpcInvocation = remoteRpcInvocation; - } - } catch (IOException e) { - LOG.warn("Could not create remote rpc invocation message. Failing rpc invocation because...", e); - throw e; - } - } - - Class returnType = method.getReturnType(); - - if (Objects.equals(returnType, Void.TYPE)) { - rpcEndpoint.tell(rpcInvocation, ActorRef.noSender()); - - result = null; - } else if (Objects.equals(returnType,CompletableFuture.class)) { - // execute an asynchronous call - result = FutureUtils.toJava(Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds())); - } else { - // execute a synchronous call - CompletableFuture futureResult = FutureUtils.toJava( - Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout.toMilliseconds())); - - result = futureResult.get(futureTimeout.getSize(), futureTimeout.getUnit()); - } + result = invokeRpc(method, args); } return result; @@ -171,7 +133,7 @@ public ActorRef getRpcEndpoint() { @Override public void runAsync(Runnable runnable) { - scheduleRunAsync(runnable, 0); + scheduleRunAsync(runnable, 0L); } @Override @@ -181,7 +143,7 @@ public void scheduleRunAsync(Runnable runnable, long delayMillis) { if (isLocal) { long atTimeNanos = delayMillis == 0 ? 0 : System.nanoTime() + (delayMillis * 1_000_000); - rpcEndpoint.tell(new RunAsync(runnable, atTimeNanos), ActorRef.noSender()); + tell(new RunAsync(runnable, atTimeNanos)); } else { throw new RuntimeException("Trying to send a Runnable to a remote actor at " + rpcEndpoint.path() + ". This is not supported."); @@ -192,9 +154,9 @@ public void scheduleRunAsync(Runnable runnable, long delayMillis) { public CompletableFuture callAsync(Callable callable, Time callTimeout) { if(isLocal) { @SuppressWarnings("unchecked") - scala.concurrent.Future resultFuture = (scala.concurrent.Future) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout.toMilliseconds()); + CompletableFuture resultFuture = (CompletableFuture) ask(new CallAsync(callable), callTimeout); - return FutureUtils.toJava(resultFuture); + return resultFuture; } else { throw new RuntimeException("Trying to send a Callable to a remote actor at " + rpcEndpoint.path() + ". This is not supported."); @@ -211,6 +173,88 @@ public void stop() { rpcEndpoint.tell(Processing.STOP, ActorRef.noSender()); } + // ------------------------------------------------------------------------ + // Private methods + // ------------------------------------------------------------------------ + + /** + * Invokes a RPC method by sending the RPC invocation details to the rpc endpoint. + * + * @param method to call + * @param args of the method call + * @return result of the RPC + * @throws Exception if the RPC invocation fails + */ + private Object invokeRpc(Method method, Object[] args) throws Exception { + String methodName = method.getName(); + Class[] parameterTypes = method.getParameterTypes(); + Annotation[][] parameterAnnotations = method.getParameterAnnotations(); + Time futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout); + + final RpcInvocation rpcInvocation = createRpcInvocationMessage(methodName, parameterTypes, args); + + Class returnType = method.getReturnType(); + + final Object result; + + if (Objects.equals(returnType, Void.TYPE)) { + tell(rpcInvocation); + + result = null; + } else if (Objects.equals(returnType,CompletableFuture.class)) { + // execute an asynchronous call + result = ask(rpcInvocation, futureTimeout); + } else { + // execute a synchronous call + CompletableFuture futureResult = ask(rpcInvocation, futureTimeout); + + result = futureResult.get(futureTimeout.getSize(), futureTimeout.getUnit()); + } + + return result; + } + + /** + * Create the RpcInvocation message for the given RPC. + * + * @param methodName of the RPC + * @param parameterTypes of the RPC + * @param args of the RPC + * @return RpcInvocation message which encapsulates the RPC details + * @throws IOException if we cannot serialize the RPC invocation parameters + */ + protected RpcInvocation createRpcInvocationMessage( + final String methodName, + final Class[] parameterTypes, + final Object[] args) throws IOException { + final RpcInvocation rpcInvocation; + + if (isLocal) { + rpcInvocation = new LocalRpcInvocation( + methodName, + parameterTypes, + args); + } else { + try { + RemoteRpcInvocation remoteRpcInvocation = new RemoteRpcInvocation( + methodName, + parameterTypes, + args); + + if (remoteRpcInvocation.getSize() > maximumFramesize) { + throw new IOException("The rpc invocation size exceeds the maximum akka framesize."); + } else { + rpcInvocation = remoteRpcInvocation; + } + } catch (IOException e) { + LOG.warn("Could not create remote rpc invocation message. Failing rpc invocation because...", e); + throw e; + } + } + + return rpcInvocation; + } + // ------------------------------------------------------------------------ // Helper methods // ------------------------------------------------------------------------ @@ -262,6 +306,28 @@ private static boolean isRpcTimeout(Annotation[] annotations) { return false; } + /** + * Sends the message to the RPC endpoint. + * + * @param message to send to the RPC endpoint. + */ + protected void tell(Object message) { + rpcEndpoint.tell(message, ActorRef.noSender()); + } + + /** + * Sends the message to the RPC endpoint and returns a future containing + * its response. + * + * @param message to send to the RPC endpoint + * @param timeout time to wait until the response future is failed with a {@link TimeoutException} + * @return Response future + */ + protected CompletableFuture ask(Object message, Time timeout) { + return FutureUtils.toJava( + Patterns.ask(rpcEndpoint, message, timeout.toMilliseconds())); + } + @Override public String getAddress() { return address; 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 d51607ecb5d97..74c1509757a32 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 @@ -21,19 +21,20 @@ import akka.actor.ActorRef; import akka.actor.Status; import akka.actor.UntypedActor; -import akka.japi.Procedure; import akka.pattern.Patterns; import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; -import org.apache.flink.runtime.rpc.akka.messages.CallAsync; -import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation; +import org.apache.flink.runtime.rpc.akka.exceptions.AkkaUnknownMessageException; +import org.apache.flink.runtime.rpc.messages.CallAsync; +import org.apache.flink.runtime.rpc.messages.ControlMessage; +import org.apache.flink.runtime.rpc.messages.LocalRpcInvocation; import org.apache.flink.runtime.rpc.akka.messages.Processing; -import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation; -import org.apache.flink.runtime.rpc.akka.messages.RunAsync; +import org.apache.flink.runtime.rpc.messages.RpcInvocation; +import org.apache.flink.runtime.rpc.messages.RunAsync; -import org.apache.flink.runtime.rpc.akka.messages.Shutdown; +import org.apache.flink.runtime.rpc.messages.Shutdown; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; @@ -70,10 +71,10 @@ */ class AkkaRpcActor extends UntypedActor { - private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActor.class); + protected final Logger log = LoggerFactory.getLogger(getClass()); /** the endpoint to invoke the methods on */ - private final T rpcEndpoint; + protected final T rpcEndpoint; /** the helper that tracks whether calls come from the main thread */ private final MainThreadValidatorUtil mainThreadValidator; @@ -110,48 +111,63 @@ public void postStop() throws Exception { @Override public void onReceive(final Object message) { if (message.equals(Processing.START)) { - getContext().become(new Procedure() { - @Override - public void apply(Object msg) throws Exception { + getContext().become( + (Object msg) -> { if (msg.equals(Processing.STOP)) { getContext().unbecome(); } else { - handleMessage(msg); + mainThreadValidator.enterMainThread(); + + try { + if (msg instanceof ControlMessage) { + handleControlMessage(((ControlMessage) msg)); + } else { + handleMessage(msg); + } + } finally { + mainThreadValidator.exitMainThread(); + } } - } - }); + }); } else { - LOG.info("The rpc endpoint {} has not been started yet. Discarding message {} until processing is started.", + log.info("The rpc endpoint {} has not been started yet. Discarding message {} until processing is started.", rpcEndpoint.getClass().getName(), message.getClass().getName()); - if (!getSender().equals(ActorRef.noSender())) { - // fail a possible future if we have a sender - getSender().tell(new Status.Failure(new AkkaRpcException("Discard message, because " + - "the rpc endpoint has not been started yet.")), getSelf()); - } + sendErrorIfSender(new AkkaRpcException("Discard message, because " + + "the rpc endpoint has not been started yet.")); } } - private void handleMessage(Object message) { - mainThreadValidator.enterMainThread(); - try { - if (message instanceof RunAsync) { - handleRunAsync((RunAsync) message); - } else if (message instanceof CallAsync) { - handleCallAsync((CallAsync) message); - } else if (message instanceof RpcInvocation) { - handleRpcInvocation((RpcInvocation) message); - } else if (message instanceof Shutdown) { - triggerShutdown(); - } else { - LOG.warn( - "Received message of unknown type {} with value {}. Dropping this message!", - message.getClass().getName(), - message); - } - } finally { - mainThreadValidator.exitMainThread(); + private void handleControlMessage(ControlMessage controlMessage) { + if (controlMessage instanceof Shutdown) { + triggerShutdown(); + } else { + log.warn( + "Received control message of unknown type {} with value {}. Dropping this control message!", + controlMessage.getClass().getName(), + controlMessage); + + sendErrorIfSender(new AkkaUnknownMessageException("Received unknown control message " + controlMessage + + " of type " + controlMessage.getClass().getSimpleName() + '.')); + } + } + + protected void handleMessage(Object message) { + if (message instanceof RunAsync) { + handleRunAsync((RunAsync) message); + } else if (message instanceof CallAsync) { + handleCallAsync((CallAsync) message); + } else if (message instanceof RpcInvocation) { + handleRpcInvocation((RpcInvocation) message); + } else { + log.warn( + "Received message of unknown type {} with value {}. Dropping this message!", + message.getClass().getName(), + message); + + sendErrorIfSender(new AkkaUnknownMessageException("Received unknown message " + message + + " of type " + message.getClass().getSimpleName() + '.')); } } @@ -171,17 +187,17 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { rpcMethod = lookupRpcMethod(methodName, parameterTypes); } catch(ClassNotFoundException e) { - LOG.error("Could not load method arguments.", e); + log.error("Could not load method arguments.", e); RpcConnectionException rpcException = new RpcConnectionException("Could not load method arguments.", e); getSender().tell(new Status.Failure(rpcException), getSelf()); } catch (IOException e) { - LOG.error("Could not deserialize rpc invocation message.", e); + log.error("Could not deserialize rpc invocation message.", e); RpcConnectionException rpcException = new RpcConnectionException("Could not deserialize rpc invocation message.", e); getSender().tell(new Status.Failure(rpcException), getSelf()); } catch (final NoSuchMethodException e) { - LOG.error("Could not find rpc method for rpc invocation.", e); + log.error("Could not find rpc method for rpc invocation.", e); RpcConnectionException rpcException = new RpcConnectionException("Could not find rpc method for rpc invocation.", e); getSender().tell(new Status.Failure(rpcException), getSelf()); @@ -202,7 +218,7 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs()); } catch (InvocationTargetException e) { - LOG.trace("Reporting back error thrown in remote procedure {}", rpcMethod, e); + log.trace("Reporting back error thrown in remote procedure {}", rpcMethod, e); // tell the sender about the failure getSender().tell(new Status.Failure(e.getTargetException()), getSelf()); @@ -229,7 +245,7 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { } } } catch (Throwable e) { - LOG.error("Error while executing remote procedure call {}.", rpcMethod, e); + log.error("Error while executing remote procedure call {}.", rpcMethod, e); // tell the sender about the failure getSender().tell(new Status.Failure(e), getSelf()); } @@ -249,9 +265,9 @@ private void handleCallAsync(CallAsync callAsync) { "prior to sending the message. The " + callAsync.getClass().getName() + " is only supported with local communication."; - LOG.warn(result); + log.warn(result); - getSender().tell(new Status.Failure(new Exception(result)), getSelf()); + getSender().tell(new Status.Failure(new AkkaRpcException(result)), getSelf()); } else { try { Object result = callAsync.getCallable().call(); @@ -271,7 +287,7 @@ private void handleCallAsync(CallAsync callAsync) { */ private void handleRunAsync(RunAsync runAsync) { if (runAsync.getRunnable() == null) { - LOG.warn("Received a {} message with an empty runnable field. This indicates " + + log.warn("Received a {} message with an empty runnable field. This indicates " + "that this message has been serialized prior to sending the message. The " + "{} is only supported with local communication.", runAsync.getClass().getName(), @@ -286,7 +302,7 @@ private void handleRunAsync(RunAsync runAsync) { try { runAsync.getRunnable().run(); } catch (Throwable t) { - LOG.error("Caught exception while executing runnable in main thread.", t); + log.error("Caught exception while executing runnable in main thread.", t); ExceptionUtils.rethrowIfFatalErrorOrOOM(t); } } @@ -324,4 +340,15 @@ private void triggerShutdown() { private Method lookupRpcMethod(final String methodName, final Class[] parameterTypes) throws NoSuchMethodException { return rpcEndpoint.getClass().getMethod(methodName, parameterTypes); } + + /** + * Send throwable to sender if the sender is specified. + * + * @param throwable to send to the sender + */ + protected void sendErrorIfSender(Throwable throwable) { + if (!getSender().equals(ActorRef.noSender())) { + getSender().tell(new Status.Failure(throwable), getSelf()); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index ab851f6c8065b..536a7892659ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -30,15 +30,19 @@ import akka.dispatch.Mapper; import akka.pattern.Patterns; import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; +import org.apache.flink.runtime.rpc.FencedMainThreadExecutable; +import org.apache.flink.runtime.rpc.FencedRpcEndpoint; +import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcServer; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.akka.messages.Shutdown; +import org.apache.flink.runtime.rpc.messages.Shutdown; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -49,6 +53,8 @@ import javax.annotation.Nonnull; import javax.annotation.concurrent.ThreadSafe; + +import java.io.Serializable; import java.lang.reflect.InvocationHandler; import java.lang.reflect.Proxy; import java.util.HashSet; @@ -61,6 +67,7 @@ import java.util.concurrent.RunnableScheduledFuture; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -131,60 +138,44 @@ public int getPort() { // this method does not mutate state and is thus thread-safe @Override - public CompletableFuture connect(final String address, final Class clazz) { - checkState(!stopped, "RpcService is stopped"); - - LOG.debug("Try to connect to remote RPC endpoint with address {}. Returning a {} gateway.", - address, clazz.getName()); - - final ActorSelection actorSel = actorSystem.actorSelection(address); - - final scala.concurrent.Future identify = Patterns.ask(actorSel, new Identify(42), timeout.toMilliseconds()); - final scala.concurrent.Future resultFuture = identify.map(new Mapper(){ - @Override - public C checkedApply(Object obj) throws Exception { - - ActorIdentity actorIdentity = (ActorIdentity) obj; - - if (actorIdentity.getRef() == null) { - throw new RpcConnectionException("Could not connect to rpc endpoint under address " + address + '.'); - } else { - ActorRef actorRef = actorIdentity.getRef(); - - final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef); - final String hostname; - Option host = actorRef.path().address().host(); - if (host.isEmpty()) { - hostname = "localhost"; - } else { - hostname = host.get(); - } - - InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler( - address, - hostname, - actorRef, - timeout, - maximumFramesize, - null); - - // Rather than using the System ClassLoader directly, we derive the ClassLoader - // from this class . That works better in cases where Flink runs embedded and all Flink - // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader - ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader(); - - @SuppressWarnings("unchecked") - C proxy = (C) Proxy.newProxyInstance( - classLoader, - new Class[]{clazz}, - akkaInvocationHandler); + public CompletableFuture connect( + final String address, + final Class clazz) { - return proxy; - } - } - }, actorSystem.dispatcher()); + return connectInternal( + address, + clazz, + (ActorRef actorRef) -> { + Tuple2 addressHostname = extractAddressHostname(actorRef); + + return new AkkaInvocationHandler( + addressHostname.f0, + addressHostname.f1, + actorRef, + timeout, + maximumFramesize, + null); + }); + } - return FutureUtils.toJava(resultFuture); + // this method does not mutate state and is thus thread-safe + @Override + public > CompletableFuture connect(String address, F fencingToken, Class clazz) { + return connectInternal( + address, + clazz, + (ActorRef actorRef) -> { + Tuple2 addressHostname = extractAddressHostname(actorRef); + + return new FencedAkkaInvocationHandler<>( + addressHostname.f0, + addressHostname.f1, + actorRef, + timeout, + maximumFramesize, + null, + () -> fencingToken); + }); } @Override @@ -192,7 +183,14 @@ public RpcServer startServer(C rpcEndpoint) checkNotNull(rpcEndpoint, "rpc endpoint"); CompletableFuture terminationFuture = new CompletableFuture<>(); - Props akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint, terminationFuture); + final Props akkaRpcActorProps; + + if (rpcEndpoint instanceof FencedRpcEndpoint) { + akkaRpcActorProps = Props.create(FencedAkkaRpcActor.class, rpcEndpoint, terminationFuture); + } else { + akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint, terminationFuture); + } + ActorRef actorRef; synchronized (lock) { @@ -212,24 +210,40 @@ public RpcServer startServer(C rpcEndpoint) hostname = host.get(); } - InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler( - address, - hostname, - actorRef, - timeout, - maximumFramesize, - terminationFuture); + Set> implementedRpcGateways = new HashSet<>(RpcUtils.extractImplementedRpcGateways(rpcEndpoint.getClass())); + + implementedRpcGateways.add(RpcServer.class); + implementedRpcGateways.add(AkkaGateway.class); + + final InvocationHandler akkaInvocationHandler; + + if (rpcEndpoint instanceof FencedRpcEndpoint) { + // a FencedRpcEndpoint needs a FencedAkkaInvocationHandler + akkaInvocationHandler = new FencedAkkaInvocationHandler<>( + address, + hostname, + actorRef, + timeout, + maximumFramesize, + terminationFuture, + ((FencedRpcEndpoint) rpcEndpoint)::getFencingToken); + + implementedRpcGateways.add(FencedMainThreadExecutable.class); + } else { + akkaInvocationHandler = new AkkaInvocationHandler( + address, + hostname, + actorRef, + timeout, + maximumFramesize, + terminationFuture); + } // Rather than using the System ClassLoader directly, we derive the ClassLoader // from this class . That works better in cases where Flink runs embedded and all Flink // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader ClassLoader classLoader = getClass().getClassLoader(); - Set> implementedRpcGateways = RpcUtils.extractImplementedRpcGateways(rpcEndpoint.getClass()); - - implementedRpcGateways.add(RpcServer.class); - implementedRpcGateways.add(AkkaGateway.class); - @SuppressWarnings("unchecked") RpcServer server = (RpcServer) Proxy.newProxyInstance( classLoader, @@ -239,6 +253,33 @@ public RpcServer startServer(C rpcEndpoint) return server; } + @Override + public RpcServer fenceRpcServer(RpcServer rpcServer, F fencingToken) { + if (rpcServer instanceof AkkaGateway) { + + InvocationHandler fencedInvocationHandler = new FencedAkkaInvocationHandler<>( + rpcServer.getAddress(), + rpcServer.getHostname(), + ((AkkaGateway) rpcServer).getRpcEndpoint(), + timeout, + maximumFramesize, + null, + () -> fencingToken); + + // Rather than using the System ClassLoader directly, we derive the ClassLoader + // from this class . That works better in cases where Flink runs embedded and all Flink + // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader + ClassLoader classLoader = getClass().getClassLoader(); + + return (RpcServer) Proxy.newProxyInstance( + classLoader, + new Class[]{RpcServer.class, AkkaGateway.class}, + fencedInvocationHandler); + } else { + throw new RuntimeException("The given RpcServer must implement the AkkaGateway in order to fence it."); + } + } + @Override public void stopServer(RpcServer selfGateway) { if (selfGateway instanceof AkkaGateway) { @@ -317,6 +358,67 @@ public CompletableFuture execute(Callable callable) { return FutureUtils.toJava(scalaFuture); } + // --------------------------------------------------------------------------------------- + // Private helper methods + // --------------------------------------------------------------------------------------- + + private Tuple2 extractAddressHostname(ActorRef actorRef) { + final String actorAddress = AkkaUtils.getAkkaURL(actorSystem, actorRef); + final String hostname; + Option host = actorRef.path().address().host(); + if (host.isEmpty()) { + hostname = "localhost"; + } else { + hostname = host.get(); + } + + return Tuple2.of(actorAddress, hostname); + } + + private CompletableFuture connectInternal( + final String address, + final Class clazz, + Function invocationHandlerFactory) { + checkState(!stopped, "RpcService is stopped"); + + LOG.debug("Try to connect to remote RPC endpoint with address {}. Returning a {} gateway.", + address, clazz.getName()); + + final ActorSelection actorSel = actorSystem.actorSelection(address); + + final Future identify = Patterns.ask(actorSel, new Identify(42), timeout.toMilliseconds()); + final Future resultFuture = identify.map(new Mapper(){ + @Override + public C checkedApply(Object obj) throws Exception { + + ActorIdentity actorIdentity = (ActorIdentity) obj; + + if (actorIdentity.getRef() == null) { + throw new RpcConnectionException("Could not connect to rpc endpoint under address " + address + '.'); + } else { + ActorRef actorRef = actorIdentity.getRef(); + + InvocationHandler invocationHandler = invocationHandlerFactory.apply(actorRef); + + // Rather than using the System ClassLoader directly, we derive the ClassLoader + // from this class . That works better in cases where Flink runs embedded and all Flink + // code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader + ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader(); + + @SuppressWarnings("unchecked") + C proxy = (C) Proxy.newProxyInstance( + classLoader, + new Class[]{clazz}, + invocationHandler); + + return proxy; + } + } + }, actorSystem.dispatcher()); + + return FutureUtils.toJava(resultFuture); + } + /** * Helper class to expose the internal scheduling logic via a {@link ScheduledExecutor}. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.java new file mode 100644 index 0000000000000..9d2c2950d98f9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaInvocationHandler.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.runtime.rpc.akka; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FutureUtils; +import org.apache.flink.runtime.rpc.FencedMainThreadExecutable; +import org.apache.flink.runtime.rpc.FencedRpcEndpoint; +import org.apache.flink.runtime.rpc.FencedRpcGateway; +import org.apache.flink.runtime.rpc.messages.CallAsync; +import org.apache.flink.runtime.rpc.messages.FencedMessage; +import org.apache.flink.runtime.rpc.messages.LocalFencedMessage; +import org.apache.flink.runtime.rpc.messages.RemoteFencedMessage; +import org.apache.flink.runtime.rpc.messages.RunAsync; +import org.apache.flink.runtime.rpc.messages.UnfencedMessage; +import org.apache.flink.util.Preconditions; + +import akka.actor.ActorRef; +import akka.pattern.Patterns; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.lang.reflect.Method; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Fenced extension of the {@link AkkaInvocationHandler}. This invocation handler will be used in combination + * with the {@link FencedRpcEndpoint}. The fencing is done by wrapping all messages in a {@link FencedMessage}. + * + * @param type of the fencing token + */ +public class FencedAkkaInvocationHandler extends AkkaInvocationHandler implements FencedMainThreadExecutable, FencedRpcGateway { + + private final Supplier fencingTokenSupplier; + + public FencedAkkaInvocationHandler( + String address, + String hostname, + ActorRef rpcEndpoint, + Time timeout, + long maximumFramesize, + @Nullable CompletableFuture terminationFuture, + Supplier fencingTokenSupplier) { + super(address, hostname, rpcEndpoint, timeout, maximumFramesize, terminationFuture); + + this.fencingTokenSupplier = Preconditions.checkNotNull(fencingTokenSupplier); + } + + @Override + public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { + Class declaringClass = method.getDeclaringClass(); + + if (declaringClass.equals(FencedMainThreadExecutable.class) || + declaringClass.equals(FencedRpcGateway.class)) { + return method.invoke(this, args); + } else { + return super.invoke(proxy, method, args); + } + } + + @Override + public void runAsyncWithoutFencing(Runnable runnable) { + checkNotNull(runnable, "runnable"); + + if (isLocal) { + getRpcEndpoint().tell( + new UnfencedMessage<>(new RunAsync(runnable, 0L)), ActorRef.noSender()); + } else { + throw new RuntimeException("Trying to send a Runnable to a remote actor at " + + getRpcEndpoint().path() + ". This is not supported."); + } + } + + @Override + public CompletableFuture callAsyncWithoutFencing(Callable callable, Time timeout) { + checkNotNull(callable, "callable"); + checkNotNull(timeout, "timeout"); + + if (isLocal) { + @SuppressWarnings("unchecked") + CompletableFuture resultFuture = (CompletableFuture) FutureUtils.toJava( + Patterns.ask( + getRpcEndpoint(), + new UnfencedMessage<>(new CallAsync(callable)), + timeout.toMilliseconds())); + + return resultFuture; + } else { + throw new RuntimeException("Trying to send a Runnable to a remote actor at " + + getRpcEndpoint().path() + ". This is not supported."); + } + } + + @Override + public void tell(Object message) { + super.tell(fenceMessage(message)); + } + + @Override + public CompletableFuture ask(Object message, Time timeout) { + return super.ask(fenceMessage(message), timeout); + } + + @Override + public F getFencingToken() { + return fencingTokenSupplier.get(); + } + + private

FencedMessage fenceMessage(P message) { + if (isLocal) { + return new LocalFencedMessage<>(fencingTokenSupplier.get(), message); + } else { + if (message instanceof Serializable) { + @SuppressWarnings("unchecked") + FencedMessage result = (FencedMessage) new RemoteFencedMessage<>(fencingTokenSupplier.get(), (Serializable) message); + + return result; + } else { + throw new RuntimeException("Trying to send a non-serializable message " + message + " to a remote " + + "RpcEndpoint. Please make sure that the message implements java.io.Serializable."); + } + } + } +} 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 new file mode 100644 index 0000000000000..b10f7deccec3c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/FencedAkkaRpcActor.java @@ -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.runtime.rpc.akka; + +import org.apache.flink.runtime.rpc.FencedRpcEndpoint; +import org.apache.flink.runtime.rpc.akka.exceptions.AkkaUnknownMessageException; +import org.apache.flink.runtime.rpc.exceptions.FencingTokenMismatchException; +import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.messages.FencedMessage; +import org.apache.flink.runtime.rpc.messages.UnfencedMessage; + +import java.io.Serializable; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; + +/** + * Fenced extension of the {@link AkkaRpcActor}. This actor will be started for {@link FencedRpcEndpoint} and is + * responsible for filtering out invalid messages with respect to the current fencing token. + * + * @param type of the fencing token + * @param type of the RpcEndpoint + */ +public class FencedAkkaRpcActor & RpcGateway> extends AkkaRpcActor { + + public FencedAkkaRpcActor(T rpcEndpoint, CompletableFuture terminationFuture) { + super(rpcEndpoint, terminationFuture); + } + + @Override + protected void handleMessage(Object message) { + if (message instanceof FencedMessage) { + @SuppressWarnings("unchecked") + FencedMessage fencedMessage = ((FencedMessage) message); + + F fencingToken = fencedMessage.getFencingToken(); + + if (Objects.equals(rpcEndpoint.getFencingToken(), fencingToken)) { + super.handleMessage(fencedMessage.getPayload()); + } else { + if (log.isDebugEnabled()) { + log.debug("Fencing token mismatch: Ignoring message {} because the fencing token {} did " + + "not match the expected fencing token {}.", message, fencingToken, rpcEndpoint.getFencingToken()); + } + + sendErrorIfSender(new FencingTokenMismatchException("Expected fencing token " + rpcEndpoint.getFencingToken() + ", actual fencing token " + fencingToken)); + } + } else if (message instanceof UnfencedMessage) { + super.handleMessage(((UnfencedMessage) message).getPayload()); + } else { + if (log.isDebugEnabled()) { + log.debug("Unknown message type: Ignoring message {} because it is neither of type {} nor {}.", + message, FencedMessage.class.getSimpleName(), UnfencedMessage.class.getSimpleName()); + } + + sendErrorIfSender(new AkkaUnknownMessageException("Unknown message type: Ignoring message " + message + + " of type " + message.getClass().getSimpleName() + " because it is neither of type " + + FencedMessage.class.getSimpleName() + " nor " + UnfencedMessage.class.getSimpleName() + '.')); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.java new file mode 100644 index 0000000000000..7504761591c14 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/exceptions/AkkaUnknownMessageException.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.rpc.akka.exceptions; + +/** + * Exception which indicates that the AkkaRpcActor has received an + * unknown message type. + */ +public class AkkaUnknownMessageException extends AkkaRpcException { + + private static final long serialVersionUID = 1691338049911020814L; + + public AkkaUnknownMessageException(String message) { + super(message); + } + + public AkkaUnknownMessageException(String message, Throwable cause) { + super(message, cause); + } + + public AkkaUnknownMessageException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java index 5c7df5dcd6021..030ff60bed6e2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java @@ -21,7 +21,7 @@ /** * Controls the processing behaviour of the {@link org.apache.flink.runtime.rpc.akka.AkkaRpcActor} */ -public enum Processing { +public enum Processing { START, // Unstashes all stashed messages and starts processing incoming messages STOP // Stop processing messages and stashes all incoming messages } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/FencingTokenMismatchException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/FencingTokenMismatchException.java new file mode 100644 index 0000000000000..9a59101a0e813 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/FencingTokenMismatchException.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.runtime.rpc.exceptions; + +import org.apache.flink.runtime.rpc.FencedRpcEndpoint; +import org.apache.flink.runtime.rpc.exceptions.RpcException; + +/** + * Exception which is thrown if the fencing tokens of a {@link FencedRpcEndpoint} do + * not match. + */ +public class FencingTokenMismatchException extends RpcException { + private static final long serialVersionUID = -500634972988881467L; + + public FencingTokenMismatchException(String message) { + super(message); + } + + public FencingTokenMismatchException(String message, Throwable cause) { + super(message, cause); + } + + public FencingTokenMismatchException(Throwable cause) { + super(cause); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/CallAsync.java similarity index 96% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/CallAsync.java index 79b7825e8a3ee..9aa7d70d3155b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/CallAsync.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka.messages; +package org.apache.flink.runtime.rpc.messages; import org.apache.flink.util.Preconditions; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/ControlMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/ControlMessage.java new file mode 100644 index 0000000000000..c16bdd75a1029 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/ControlMessage.java @@ -0,0 +1,26 @@ +/* + * 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.rpc.messages; + +/** + * Base interface for control messages which are treated separately by the RPC server + * implementation. + */ +public interface ControlMessage { +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/FencedMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/FencedMessage.java new file mode 100644 index 0000000000000..b67e564f593a4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/FencedMessage.java @@ -0,0 +1,34 @@ +/* + * 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.rpc.messages; + +import java.io.Serializable; + +/** + * Interface for fenced messages. + * + * @param type of the fencing token + * @param

type of the payload + */ +public interface FencedMessage { + + F getFencingToken(); + + P getPayload(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/LocalFencedMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/LocalFencedMessage.java new file mode 100644 index 0000000000000..248106558d3b8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/LocalFencedMessage.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.runtime.rpc.messages; + +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * Local {@link FencedMessage} implementation. This message is used when the communication + * is local and thus does not require its payload to be serializable. + * + * @param type of the fencing token + * @param

type of the payload + */ +public class LocalFencedMessage implements FencedMessage { + + private final F fencingToken; + private final P payload; + + public LocalFencedMessage(F fencingToken, P payload) { + this.fencingToken = Preconditions.checkNotNull(fencingToken); + this.payload = Preconditions.checkNotNull(payload); + } + + @Override + public F getFencingToken() { + return fencingToken; + } + + @Override + public P getPayload() { + return payload; + } + + @Override + public String toString() { + return "LocalFencedMessage(" + fencingToken + ", " + payload + ')'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/LocalRpcInvocation.java similarity index 72% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/LocalRpcInvocation.java index 97c10d71bf141..0bd06c338bcf6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/LocalRpcInvocation.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka.messages; +package org.apache.flink.runtime.rpc.messages; import org.apache.flink.util.Preconditions; @@ -31,10 +31,14 @@ public final class LocalRpcInvocation implements RpcInvocation { private final Class[] parameterTypes; private final Object[] args; + private transient String toString; + public LocalRpcInvocation(String methodName, Class[] parameterTypes, Object[] args) { this.methodName = Preconditions.checkNotNull(methodName); this.parameterTypes = Preconditions.checkNotNull(parameterTypes); this.args = args; + + toString = null; } @Override @@ -51,4 +55,25 @@ public Class[] getParameterTypes() { public Object[] getArgs() { return args; } + + @Override + public String toString() { + if (toString == null) { + StringBuilder paramTypeStringBuilder = new StringBuilder(parameterTypes.length * 5); + + if (parameterTypes.length > 0) { + paramTypeStringBuilder.append(parameterTypes[0].getSimpleName()); + + for (int i = 1; i < parameterTypes.length; i++) { + paramTypeStringBuilder + .append(", ") + .append(parameterTypes[i].getSimpleName()); + } + } + + toString = "LocalRpcInvocation(" + methodName + '(' + paramTypeStringBuilder + "))"; + } + + return toString; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteFencedMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteFencedMessage.java new file mode 100644 index 0000000000000..5cf9b98d6202a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteFencedMessage.java @@ -0,0 +1,57 @@ +/* + * 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.rpc.messages; + +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * Remote {@link FencedMessage} implementation. This message is used when the communication + * is remote and thus requires its payload to be serializable. + * + * @param type of the fencing token + * @param

type of the payload + */ +public class RemoteFencedMessage implements FencedMessage, Serializable { + private static final long serialVersionUID = 4043136067468477742L; + + private final F fencingToken; + private final P payload; + + public RemoteFencedMessage(F fencingToken, P payload) { + this.fencingToken = Preconditions.checkNotNull(fencingToken); + this.payload = Preconditions.checkNotNull(payload); + } + + @Override + public F getFencingToken() { + return fencingToken; + } + + @Override + public P getPayload() { + return payload; + } + + @Override + public String toString() { + return "RemoteFencedMessage(" + fencingToken + ", " + payload + ')'; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java similarity index 88% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java index bc26a29715c91..779d5dd82c159 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RemoteRpcInvocation.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka.messages; +package org.apache.flink.runtime.rpc.messages; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -43,6 +43,8 @@ public class RemoteRpcInvocation implements RpcInvocation, Serializable { // Transient field which is lazily initialized upon first access to the invocation data private transient RemoteRpcInvocation.MethodInvocation methodInvocation; + private transient String toString; + public RemoteRpcInvocation( final String methodName, final Class[] parameterTypes, @@ -73,6 +75,35 @@ public Object[] getArgs() throws IOException, ClassNotFoundException { return methodInvocation.getArgs(); } + @Override + public String toString() { + if (toString == null) { + + try { + Class[] parameterTypes = getParameterTypes(); + String methodName = getMethodName(); + + StringBuilder paramTypeStringBuilder = new StringBuilder(parameterTypes.length * 5); + + if (parameterTypes.length > 0) { + paramTypeStringBuilder.append(parameterTypes[0].getSimpleName()); + + for (int i = 1; i < parameterTypes.length; i++) { + paramTypeStringBuilder + .append(", ") + .append(parameterTypes[i].getSimpleName()); + } + } + + toString = "RemoteRpcInvocation(" + methodName + '(' + paramTypeStringBuilder + "))"; + } catch (IOException | ClassNotFoundException e) { + toString = "Could not deserialize RemoteRpcInvocation: " + e.getMessage(); + } + } + + return toString; + } + /** * Size (#bytes of the serialized data) of the rpc invocation message. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RpcInvocation.java similarity index 97% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RpcInvocation.java index b174c99a4d37c..4e9f6299fec67 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RpcInvocation.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka.messages; +package org.apache.flink.runtime.rpc.messages; import java.io.IOException; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RunAsync.java similarity index 95% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RunAsync.java index 4b8a0b4c6542a..2f6d867ea833a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/RunAsync.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka.messages; +package org.apache.flink.runtime.rpc.messages; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkArgument; @@ -26,7 +26,7 @@ */ public final class RunAsync { - /** The runnable to be executed. Transient, so it gets lost upon serialization */ + /** The runnable to be executed. Transient, so it gets lost upon serialization */ private final Runnable runnable; /** The delay after which the runnable should be called */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Shutdown.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/Shutdown.java similarity index 91% rename from flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Shutdown.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/Shutdown.java index c596d1248bf03..50b076caf6210 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Shutdown.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/Shutdown.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.rpc.akka.messages; +package org.apache.flink.runtime.rpc.messages; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; @@ -24,7 +24,7 @@ * Shut down message used to trigger the shut down of an AkkaRpcActor. This * message is only intended for internal use by the {@link AkkaRpcService}. */ -public final class Shutdown { +public final class Shutdown implements ControlMessage { private static Shutdown instance = new Shutdown(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/UnfencedMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/UnfencedMessage.java new file mode 100644 index 0000000000000..27867c4d0735f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/UnfencedMessage.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.rpc.messages; + +import org.apache.flink.runtime.rpc.FencedMainThreadExecutable; +import org.apache.flink.util.Preconditions; + +/** + * Wrapper class indicating a message which is not required to match the fencing token + * as it is used by the {@link FencedMainThreadExecutable} to run code in the main thread without + * a valid fencing token. This is required for operations which are not scoped by the current + * fencing token (e.g. leadership grants). + * + *

IMPORTANT: This message is only intended to be send locally. + * + * @param

type of the payload + */ +public class UnfencedMessage

{ + private final P payload; + + public UnfencedMessage(P payload) { + this.payload = Preconditions.checkNotNull(payload); + } + + public P getPayload() { + return payload; + } + + @Override + public String toString() { + return "UnfencedMessage(" + payload + ')'; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index 00762b9f0719a..f8eca1692c97f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -23,16 +23,23 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import org.apache.flink.runtime.rpc.exceptions.FencingTokenMismatchException; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Test; +import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Function; import static org.junit.Assert.*; @@ -44,6 +51,8 @@ public class AsyncCallsTest extends TestLogger { private static final ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem(); + private static final Time timeout = Time.seconds(10L); + private static final AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, Time.milliseconds(10000L)); @@ -162,6 +171,119 @@ public void run() { assertTrue("call was not properly delayed", ((stop - start) / 1_000_000) >= delay); } + /** + * Tests that async code is not executed if the fencing token changes. + */ + @Test + public void testRunAsyncWithFencing() throws Exception { + final Time shortTimeout = Time.milliseconds(100L); + final UUID newFencingToken = UUID.randomUUID(); + final CompletableFuture resultFuture = new CompletableFuture<>(); + + testRunAsync( + endpoint -> { + endpoint.runAsync( + () -> resultFuture.complete(endpoint.getFencingToken())); + + return resultFuture; + }, + newFencingToken); + + try { + resultFuture.get(shortTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + fail("The async run operation should not complete since it is filtered out due to the changed fencing token."); + } catch (TimeoutException ignored) {} + } + + /** + * Tests that code can be executed in the main thread without respecting the fencing token. + */ + @Test + public void testRunAsyncWithoutFencing() throws Exception { + final CompletableFuture resultFuture = new CompletableFuture<>(); + final UUID newFencingToken = UUID.randomUUID(); + + testRunAsync( + endpoint -> { + endpoint.runAsyncWithoutFencing( + () -> resultFuture.complete(endpoint.getFencingToken())); + return resultFuture; + }, + newFencingToken); + + assertEquals(newFencingToken, resultFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS)); + } + + /** + * Tests that async callables are not executed if the fencing token changes. + */ + @Test + public void testCallAsyncWithFencing() throws Exception { + final UUID newFencingToken = UUID.randomUUID(); + + CompletableFuture resultFuture = testRunAsync( + endpoint -> endpoint.callAsync(() -> true, timeout), + newFencingToken); + + try { + resultFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + fail("The async call operation should fail due to the changed fencing token."); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.stripExecutionException(e) instanceof FencingTokenMismatchException); + } + } + + /** + * Tests that async callables can be executed in the main thread without checking the fencing token. + */ + @Test + public void testCallAsyncWithoutFencing() throws Exception { + final UUID newFencingToken = UUID.randomUUID(); + + CompletableFuture resultFuture = testRunAsync( + endpoint -> endpoint.callAsyncWithoutFencing(() -> true, timeout), + newFencingToken); + + assertTrue(resultFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS)); + } + + private static CompletableFuture testRunAsync(Function> runAsyncCall, UUID newFencingToken) throws Exception { + final UUID initialFencingToken = UUID.randomUUID(); + final OneShotLatch enterSetNewFencingToken = new OneShotLatch(); + final OneShotLatch triggerSetNewFencingToken = new OneShotLatch(); + final FencedTestEndpoint fencedTestEndpoint = new FencedTestEndpoint( + akkaRpcService, + initialFencingToken, + enterSetNewFencingToken, + triggerSetNewFencingToken); + final FencedTestGateway fencedTestGateway = fencedTestEndpoint.getSelfGateway(FencedTestGateway.class); + + try { + fencedTestEndpoint.start(); + + CompletableFuture newFencingTokenFuture = fencedTestGateway.setNewFencingToken(newFencingToken, timeout); + + assertFalse(newFencingTokenFuture.isDone()); + + assertEquals(initialFencingToken, fencedTestEndpoint.getFencingToken()); + + CompletableFuture result = runAsyncCall.apply(fencedTestEndpoint); + + enterSetNewFencingToken.await(); + + triggerSetNewFencingToken.trigger(); + + newFencingTokenFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + return result; + } finally { + fencedTestEndpoint.shutDown(); + fencedTestEndpoint.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + } + // ------------------------------------------------------------------------ // test RPC endpoint // ------------------------------------------------------------------------ @@ -209,4 +331,39 @@ public boolean hasConcurrentAccess() { return concurrentAccess; } } + + public interface FencedTestGateway extends FencedRpcGateway { + CompletableFuture setNewFencingToken(UUID fencingToken, @RpcTimeout Time timeout); + } + + public static class FencedTestEndpoint extends FencedRpcEndpoint implements FencedTestGateway { + + private final OneShotLatch enteringSetNewFencingToken; + private final OneShotLatch triggerSetNewFencingToken; + + protected FencedTestEndpoint( + RpcService rpcService, + UUID initialFencingToken, + OneShotLatch enteringSetNewFencingToken, + OneShotLatch triggerSetNewFencingToken) { + super(rpcService, initialFencingToken); + + this.enteringSetNewFencingToken = enteringSetNewFencingToken; + this.triggerSetNewFencingToken = triggerSetNewFencingToken; + } + + @Override + public CompletableFuture setNewFencingToken(UUID fencingToken, Time timeout) { + enteringSetNewFencingToken.trigger(); + try { + triggerSetNewFencingToken.await(); + } catch (InterruptedException e) { + throw new RuntimeException("TriggerSetNewFencingToken OneShotLatch was interrupted."); + } + + setFencingToken(fencingToken); + + return CompletableFuture.completedFuture(Acknowledge.get()); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java new file mode 100644 index 0000000000000..62d5354fd51be --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java @@ -0,0 +1,344 @@ +/* + * 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.rpc; + +import org.apache.flink.api.common.time.Time; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.concurrent.FlinkFutureException; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.exceptions.FencingTokenMismatchException; +import org.apache.flink.runtime.rpc.exceptions.RpcException; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.TestLogger; + +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class FencedRpcEndpointTest extends TestLogger { + + private static final Time timeout = Time.seconds(10L); + private static RpcService rpcService; + + @BeforeClass + public static void setup() { + rpcService = new TestingRpcService(); + } + + @AfterClass + public static void teardown() throws ExecutionException, InterruptedException, TimeoutException { + if (rpcService != null) { + rpcService.stopService(); + rpcService.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + } + + /** + * Tests that the fencing token can be retrieved from the FencedRpcEndpoint and self + * FencedRpcGateway. Moreover it tests that you can only set the fencing token from + * the main thread. + */ + @Test + public void testFencingTokenSetting() throws Exception { + final UUID initialFencingToken = UUID.randomUUID(); + final String value = "foobar"; + FencedTestingEndpoint fencedTestingEndpoint = new FencedTestingEndpoint(rpcService, initialFencingToken, value); + FencedTestingGateway fencedTestingGateway = fencedTestingEndpoint.getSelfGateway(FencedTestingGateway.class); + FencedTestingGateway fencedGateway = fencedTestingEndpoint.getSelfGateway(FencedTestingGateway.class); + + try { + fencedTestingEndpoint.start(); + + assertEquals(initialFencingToken, fencedGateway.getFencingToken()); + assertEquals(initialFencingToken, fencedTestingEndpoint.getFencingToken()); + + final UUID newFencingToken = UUID.randomUUID(); + + try { + fencedTestingEndpoint.setFencingToken(newFencingToken); + fail("Fencing token can only be set from within the main thread."); + } catch (AssertionError ignored) { + // expected to fail + } + + assertEquals(initialFencingToken, fencedTestingEndpoint.getFencingToken()); + + CompletableFuture setFencingFuture = fencedTestingGateway.rpcSetFencingToken(newFencingToken, timeout); + + // wait for the completion of the set fencing token operation + setFencingFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + // self gateway should adapt its fencing token + assertEquals(newFencingToken, fencedGateway.getFencingToken()); + assertEquals(newFencingToken, fencedTestingEndpoint.getFencingToken()); + } finally { + fencedTestingEndpoint.shutDown(); + fencedTestingEndpoint.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + } + + /** + * Tests that messages with the wrong fencing token are filtered out. + */ + @Test + public void testFencing() throws Exception { + final UUID initialFencingToken = UUID.randomUUID(); + final UUID wrongFencingToken = UUID.randomUUID(); + final String value = "barfoo"; + FencedTestingEndpoint fencedTestingEndpoint = new FencedTestingEndpoint(rpcService, initialFencingToken, value); + + try { + fencedTestingEndpoint.start(); + + final FencedTestingGateway properFencedGateway = rpcService.connect(fencedTestingEndpoint.getAddress(), initialFencingToken, FencedTestingGateway.class) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + final FencedTestingGateway wronglyFencedGateway = rpcService.connect(fencedTestingEndpoint.getAddress(), wrongFencingToken, FencedTestingGateway.class) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + assertEquals(value, properFencedGateway.foobar(timeout).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS)); + + try { + wronglyFencedGateway.foobar(timeout).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + fail("This should fail since we have the wrong fencing token."); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.stripExecutionException(e) instanceof FencingTokenMismatchException); + } + + final UUID newFencingToken = UUID.randomUUID(); + + CompletableFuture newFencingTokenFuture = properFencedGateway.rpcSetFencingToken(newFencingToken, timeout); + + // wait for the new fencing token to be set + newFencingTokenFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + // this should no longer work because of the new fencing token + try { + properFencedGateway.foobar(timeout).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + fail("This should fail since we have the wrong fencing token by now."); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.stripExecutionException(e) instanceof FencingTokenMismatchException); + } + + } finally { + fencedTestingEndpoint.shutDown(); + fencedTestingEndpoint.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + } + + /** + * Tests that the self gateway always uses the current fencing token whereas the remote + * gateway has a fixed fencing token. + */ + @Test + public void testRemoteAndSelfGateways() throws Exception { + final UUID initialFencingToken = UUID.randomUUID(); + final UUID newFencingToken = UUID.randomUUID(); + final String value = "foobar"; + + final FencedTestingEndpoint fencedTestingEndpoint = new FencedTestingEndpoint(rpcService, initialFencingToken, value); + + try { + fencedTestingEndpoint.start(); + + FencedTestingGateway selfGateway = fencedTestingEndpoint.getSelfGateway(FencedTestingGateway.class); + FencedTestingGateway remoteGateway = rpcService.connect(fencedTestingEndpoint.getAddress(), initialFencingToken, FencedTestingGateway.class) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + assertEquals(initialFencingToken, selfGateway.getFencingToken()); + assertEquals(initialFencingToken, remoteGateway.getFencingToken()); + + assertEquals(value, selfGateway.foobar(timeout).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS)); + assertEquals(value, remoteGateway.foobar(timeout).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS)); + + CompletableFuture newFencingTokenFuture = selfGateway.rpcSetFencingToken(newFencingToken, timeout); + + // wait for the new fencing token to be set + newFencingTokenFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + assertEquals(newFencingToken, selfGateway.getFencingToken()); + assertNotEquals(newFencingToken, remoteGateway.getFencingToken()); + + assertEquals(value, selfGateway.foobar(timeout).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS)); + + try { + remoteGateway.foobar(timeout).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + fail("This should have failed because we don't have the right fencing token."); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.stripExecutionException(e) instanceof FencingTokenMismatchException); + } + } finally { + fencedTestingEndpoint.shutDown(); + fencedTestingEndpoint.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + } + + /** + * Tests that call via the MainThreadExecutor fail after the fencing token changes. + */ + @Test + public void testMainThreadExecutorUnderChangingFencingToken() throws Exception { + final Time shortTimeout = Time.milliseconds(100L); + final UUID initialFencingToken = UUID.randomUUID(); + final String value = "foobar"; + final FencedTestingEndpoint fencedTestingEndpoint = new FencedTestingEndpoint(rpcService, initialFencingToken, value); + + try { + fencedTestingEndpoint.start(); + + FencedTestingGateway selfGateway = fencedTestingEndpoint.getSelfGateway(FencedTestingGateway.class); + + CompletableFuture mainThreadExecutorComputation = selfGateway.triggerMainThreadExecutorComputation(timeout); + + // we know that subsequent calls on the same gateway are executed sequentially + // therefore, we know that the change fencing token call is executed after the trigger MainThreadExecutor + // computation + final UUID newFencingToken = UUID.randomUUID(); + CompletableFuture newFencingTokenFuture = selfGateway.rpcSetFencingToken(newFencingToken, timeout); + + newFencingTokenFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + // trigger the computation + CompletableFuture triggerFuture = selfGateway.triggerComputationLatch(timeout); + + triggerFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + // wait for the main thread executor computation to fail + try { + mainThreadExecutorComputation.get(shortTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); + fail("The MainThreadExecutor computation should be able to complete because it was filtered out leading to a timeout exception."); + } catch (TimeoutException ignored) { + // as predicted + } + + } finally { + fencedTestingEndpoint.shutDown(); + fencedTestingEndpoint.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + } + + /** + * Tests that all calls from an unfenced remote gateway are ignored and that one cannot obtain + * the fencing token from such a gateway. + */ + @Test + public void testUnfencedRemoteGateway() throws Exception { + final UUID initialFencingToken = UUID.randomUUID(); + final String value = "foobar"; + + final FencedTestingEndpoint fencedTestingEndpoint = new FencedTestingEndpoint(rpcService, initialFencingToken, value); + + try { + fencedTestingEndpoint.start(); + + FencedTestingGateway unfencedGateway = rpcService.connect(fencedTestingEndpoint.getAddress(), FencedTestingGateway.class) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + try { + unfencedGateway.foobar(timeout).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + fail("This should have failed because we have an unfenced gateway."); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.stripExecutionException(e) instanceof RpcException); + } + + try { + unfencedGateway.getFencingToken(); + fail("We should not be able to call getFencingToken on an unfenced gateway."); + } catch (UnsupportedOperationException ignored) { + // we should not be able to call getFencingToken on an unfenced gateway + } + } finally { + fencedTestingEndpoint.shutDown(); + fencedTestingEndpoint.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } + } + + public interface FencedTestingGateway extends FencedRpcGateway { + CompletableFuture foobar(@RpcTimeout Time timeout); + + CompletableFuture rpcSetFencingToken(UUID fencingToken, @RpcTimeout Time timeout); + + CompletableFuture triggerMainThreadExecutorComputation(@RpcTimeout Time timeout); + + CompletableFuture triggerComputationLatch(@RpcTimeout Time timeout); + } + + private static class FencedTestingEndpoint extends FencedRpcEndpoint implements FencedTestingGateway { + + private final OneShotLatch computationLatch; + + private final String value; + + protected FencedTestingEndpoint(RpcService rpcService, UUID initialFencingToken, String value) { + super(rpcService, initialFencingToken); + + computationLatch = new OneShotLatch(); + + this.value = value; + } + + @Override + public CompletableFuture foobar(Time timeout) { + return CompletableFuture.completedFuture(value); + } + + @Override + public CompletableFuture rpcSetFencingToken(UUID fencingToken, Time timeout) { + setFencingToken(fencingToken); + + return CompletableFuture.completedFuture(Acknowledge.get()); + } + + @Override + public CompletableFuture triggerMainThreadExecutorComputation(Time timeout) { + return CompletableFuture.supplyAsync( + () -> { + try { + computationLatch.await(); + } catch (InterruptedException e) { + throw new FlinkFutureException("Waiting on latch failed.", e); + } + + return value; + }, + getRpcService().getExecutor()) + .thenApplyAsync( + (String v) -> Acknowledge.get(), + getMainThreadExecutor()); + } + + @Override + public CompletableFuture triggerComputationLatch(Time timeout) { + computationLatch.trigger(); + + return CompletableFuture.completedFuture(Acknowledge.get()); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java index 14cf35a4d2a4e..4b9f3977fa6c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; +import java.io.Serializable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -104,7 +105,27 @@ public CompletableFuture connect(String address, Class return FutureUtils.completedExceptionally(new Exception("Gateway registered under " + address + " is not of type " + clazz)); } } else { - return FutureUtils.completedExceptionally(new Exception("No gateway registered under " + address + '.')); + return super.connect(address, clazz); + } + } + + @Override + public > CompletableFuture connect( + String address, + F fencingToken, + Class clazz) { + RpcGateway gateway = registeredConnections.get(address); + + if (gateway != null) { + if (clazz.isAssignableFrom(gateway.getClass())) { + @SuppressWarnings("unchecked") + C typedGateway = (C) gateway; + return CompletableFuture.completedFuture(typedGateway); + } else { + return FutureUtils.completedExceptionally(new Exception("Gateway registered under " + address + " is not of type " + clazz)); + } + } else { + return super.connect(address, fencingToken, clazz); } } From ab1fbfdfe6c1f2b6885710f98b9480cb90396ac0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 10 Aug 2017 10:56:12 +0200 Subject: [PATCH 083/129] [FLINK-7409] [web] Make WebRuntimeMonitor reactive This commit changes the behaviour of the WebRuntimeMonitor to not longer block serving threads by waiting on the result of futures. Instead the RequestHandler now returns a CompletableFuture which is written out to the Netty channel upon completion. This will improve the performance of our WebRuntimeMonitor. This closes #4527. --- .../RetryRejectedExecutionFailureHandler.java | 2 +- .../org/apache/flink/util/ExceptionUtils.java | 13 +- .../webmonitor/ExecutionGraphHolder.java | 16 +- .../webmonitor/RuntimeMonitorHandler.java | 60 ++-- .../runtime/webmonitor/WebRuntimeMonitor.java | 90 +++--- .../AbstractExecutionGraphRequestHandler.java | 35 ++- .../AbstractJobVertexRequestHandler.java | 10 +- .../handlers/AbstractJsonRequestHandler.java | 33 ++- .../AbstractSubtaskAttemptRequestHandler.java | 22 +- .../AbstractSubtaskRequestHandler.java | 18 +- .../handlers/ClusterOverviewHandler.java | 61 +++-- .../handlers/CurrentJobIdsHandler.java | 101 +++---- .../handlers/CurrentJobsOverviewHandler.java | 89 +++--- .../handlers/DashboardConfigHandler.java | 9 +- .../handlers/JarAccessDeniedHandler.java | 10 +- .../webmonitor/handlers/JarActionHandler.java | 4 +- .../webmonitor/handlers/JarDeleteHandler.java | 62 +++-- .../webmonitor/handlers/JarListHandler.java | 157 ++++++----- .../webmonitor/handlers/JarPlanHandler.java | 43 +-- .../webmonitor/handlers/JarRunHandler.java | 59 ++-- .../webmonitor/handlers/JarUploadHandler.java | 51 ++-- .../handlers/JobAccumulatorsHandler.java | 19 +- .../handlers/JobCancellationHandler.java | 38 ++- .../JobCancellationWithSavepointHandlers.java | 158 +++++------ .../webmonitor/handlers/JobConfigHandler.java | 20 +- .../handlers/JobDetailsHandler.java | 19 +- .../handlers/JobExceptionsHandler.java | 20 +- .../handlers/JobManagerConfigHandler.java | 60 ++-- .../webmonitor/handlers/JobPlanHandler.java | 10 +- .../handlers/JobStoppingHandler.java | 38 ++- .../JobVertexAccumulatorsHandler.java | 20 +- .../JobVertexBackPressureHandler.java | 17 +- .../handlers/JobVertexDetailsHandler.java | 19 +- .../JobVertexTaskManagersHandler.java | 19 +- .../webmonitor/handlers/RequestHandler.java | 9 +- .../SubtaskCurrentAttemptDetailsHandler.java | 8 +- ...skExecutionAttemptAccumulatorsHandler.java | 19 +- ...SubtaskExecutionAttemptDetailsHandler.java | 19 +- .../SubtasksAllAccumulatorsHandler.java | 19 +- .../handlers/SubtasksTimesHandler.java | 19 +- .../handlers/TaskManagersHandler.java | 256 +++++++++--------- .../checkpoints/CheckpointConfigHandler.java | 19 +- .../CheckpointStatsDetailsHandler.java | 63 +++-- ...CheckpointStatsDetailsSubtasksHandler.java | 33 ++- .../checkpoints/CheckpointStatsHandler.java | 19 +- .../metrics/AbstractMetricsHandler.java | 27 +- .../metrics/JobManagerMetricsHandler.java | 5 +- .../webmonitor/metrics/JobMetricsHandler.java | 5 +- .../metrics/JobVertexMetricsHandler.java | 5 +- .../metrics/TaskManagerMetricsHandler.java | 5 +- .../handlers/ClusterOverviewHandlerTest.java | 3 +- .../handlers/CurrentJobIdsHandlerTest.java | 3 +- .../CurrentJobsOverviewHandlerTest.java | 7 +- .../handlers/DashboardConfigHandlerTest.java | 3 +- .../handlers/JarAccessDeniedHandlerTest.java | 4 +- .../handlers/JarDeleteHandlerTest.java | 4 +- .../handlers/JarListHandlerTest.java | 4 +- .../handlers/JarPlanHandlerTest.java | 4 +- .../handlers/JarRunHandlerTest.java | 3 +- .../handlers/JarUploadHandlerTest.java | 4 +- .../handlers/JobAccumulatorsHandlerTest.java | 3 +- .../handlers/JobCancellationHandlerTest.java | 3 +- ...CancellationWithSavepointHandlersTest.java | 24 +- .../handlers/JobConfigHandlerTest.java | 3 +- .../handlers/JobDetailsHandlerTest.java | 3 +- .../handlers/JobExceptionsHandlerTest.java | 3 +- .../handlers/JobManagerConfigHandlerTest.java | 4 +- .../handlers/JobPlanHandlerTest.java | 3 +- .../handlers/JobStoppingHandlerTest.java | 3 +- .../JobVertexAccumulatorsHandlerTest.java | 3 +- .../JobVertexBackPressureHandlerTest.java | 12 +- .../handlers/JobVertexDetailsHandlerTest.java | 3 +- .../JobVertexTaskManagersHandlerTest.java | 3 +- ...btaskCurrentAttemptDetailsHandlerTest.java | 3 +- ...ecutionAttemptAccumulatorsHandlerTest.java | 3 +- ...askExecutionAttemptDetailsHandlerTest.java | 3 +- .../SubtasksAllAccumulatorsHandlerTest.java | 3 +- .../handlers/SubtasksTimesHandlerTest.java | 3 +- .../handlers/TaskManagersHandlerTest.java | 3 +- .../CheckpointConfigHandlerTest.java | 15 +- .../CheckpointStatsDetailsHandlerTest.java | 19 +- .../CheckpointStatsHandlerTest.java | 7 +- ...ckpointStatsSubtaskDetailsHandlerTest.java | 31 ++- .../metrics/AbstractMetricsHandlerTest.java | 20 +- .../metrics/JobManagerMetricsHandlerTest.java | 6 +- .../metrics/JobMetricsHandlerTest.java | 6 +- .../metrics/JobVertexMetricsHandlerTest.java | 6 +- .../TaskManagerMetricsHandlerTest.java | 6 +- .../flink/runtime/jobmanager/JobManager.scala | 2 +- .../taskexecutor/TaskExecutorTest.java | 2 +- 90 files changed, 1278 insertions(+), 901 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java index 9380959934cbb..370625714e96a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java @@ -36,7 +36,7 @@ public class RetryRejectedExecutionFailureHandler implements ActionRequestFailur @Override public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { - if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { indexer.add(action); } else { // rethrow all other failures 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 9c8907ba9b8fa..d141ecb1dae34 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 @@ -30,6 +30,7 @@ import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; +import java.util.Optional; import java.util.concurrent.ExecutionException; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -276,27 +277,27 @@ else if (t instanceof Error) { } /** - * Checks whether a throwable chain contains a specific type of exception. + * Checks whether a throwable chain contains a specific type of exception and returns it. * * @param throwable the throwable chain to check. * @param searchType the type of exception to search for in the chain. - * @return True, if the searched type is nested in the throwable, false otherwise. + * @return Optional throwable of the requested type if available, otherwise empty */ - public static boolean containsThrowable(Throwable throwable, Class searchType) { + public static Optional findThrowable(Throwable throwable, Class searchType) { if (throwable == null || searchType == null) { - return false; + return Optional.empty(); } Throwable t = throwable; while (t != null) { if (searchType.isAssignableFrom(t.getClass())) { - return true; + return Optional.of(t); } else { t = t.getCause(); } } - return false; + return Optional.empty(); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java index 739b375b6b9c4..8a9696948e997 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/ExecutionGraphHolder.java @@ -31,7 +31,6 @@ import java.util.Optional; import java.util.WeakHashMap; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -65,26 +64,23 @@ public ExecutionGraphHolder(Time timeout) { * * @param jid jobID of the execution graph to be retrieved * @return Optional ExecutionGraph if it has been retrievable, empty if there has been no ExecutionGraph - * @throws Exception if the ExecutionGraph retrieval failed. */ - public Optional getExecutionGraph(JobID jid, JobManagerGateway jobManagerGateway) throws Exception { + public CompletableFuture> getExecutionGraph(JobID jid, JobManagerGateway jobManagerGateway) { AccessExecutionGraph cached = cache.get(jid); if (cached != null) { if (cached.getState() == JobStatus.SUSPENDED) { cache.remove(jid); } else { - return Optional.of(cached); + return CompletableFuture.completedFuture(Optional.of(cached)); } } CompletableFuture> executionGraphFuture = jobManagerGateway.requestJob(jid, timeout); - Optional result = executionGraphFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + executionGraphFuture.thenAcceptAsync( + optExecutionGraph -> + optExecutionGraph.ifPresent(executionGraph -> cache.put(jid, executionGraph))); - return result.map((executionGraph) -> { - cache.put(jid, executionGraph); - - return executionGraph; - }); + return executionGraphFuture; } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java index 35d13ddc6e1c4..63055379df84d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/RuntimeMonitorHandler.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.webmonitor.handlers.RequestHandler; import org.apache.flink.runtime.webmonitor.retriever.JobManagerRetriever; @@ -45,6 +46,7 @@ import java.nio.charset.Charset; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -88,7 +90,7 @@ public String[] getPaths() { @Override protected void respondAsLeader(ChannelHandlerContext ctx, Routed routed, JobManagerGateway jobManagerGateway) { - FullHttpResponse response; + CompletableFuture responseFuture; try { // we only pass the first element in the list to the handlers. @@ -106,29 +108,41 @@ protected void respondAsLeader(ChannelHandlerContext ctx, Routed routed, JobMana queryParams.put(WEB_MONITOR_ADDRESS_KEY, (httpsEnabled ? "https://" : "http://") + address.getHostName() + ":" + address.getPort()); - response = handler.handleRequest(pathParams, queryParams, jobManagerGateway); + responseFuture = handler.handleRequest(pathParams, queryParams, jobManagerGateway); + } catch (Exception e) { + responseFuture = FutureUtils.completedExceptionally(e); } - catch (NotFoundException e) { - // this should result in a 404 error code (not found) - ByteBuf message = e.getMessage() == null ? Unpooled.buffer(0) - : Unpooled.wrappedBuffer(e.getMessage().getBytes(ENCODING)); - response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NOT_FOUND, message); - response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=" + ENCODING.name()); - response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes()); - LOG.debug("Error while handling request", e); - } - catch (Exception e) { - byte[] bytes = ExceptionUtils.stringifyException(e).getBytes(ENCODING); - response = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, - HttpResponseStatus.INTERNAL_SERVER_ERROR, Unpooled.wrappedBuffer(bytes)); - response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=" + ENCODING.name()); - response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes()); - - LOG.debug("Error while handling request", e); - } - - response.headers().set(HttpHeaders.Names.ACCESS_CONTROL_ALLOW_ORIGIN, allowOrigin); - KeepAliveWrite.flush(ctx, routed.request(), response); + responseFuture.whenComplete( + (FullHttpResponse httpResponse, Throwable throwable) -> { + final FullHttpResponse finalResponse; + + if (throwable != null) { + LOG.debug("Error while handling request.", throwable); + + Optional optNotFound = ExceptionUtils.findThrowable(throwable, NotFoundException.class); + + if (optNotFound.isPresent()) { + // this should result in a 404 error code (not found) + Throwable e = optNotFound.get(); + ByteBuf message = e.getMessage() == null ? Unpooled.buffer(0) + : Unpooled.wrappedBuffer(e.getMessage().getBytes(ENCODING)); + finalResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.NOT_FOUND, message); + finalResponse.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=" + ENCODING.name()); + finalResponse.headers().set(HttpHeaders.Names.CONTENT_LENGTH, finalResponse.content().readableBytes()); + } else { + byte[] bytes = ExceptionUtils.stringifyException(throwable).getBytes(ENCODING); + finalResponse = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, + HttpResponseStatus.INTERNAL_SERVER_ERROR, Unpooled.wrappedBuffer(bytes)); + finalResponse.headers().set(HttpHeaders.Names.CONTENT_TYPE, "text/plain; charset=" + ENCODING.name()); + finalResponse.headers().set(HttpHeaders.Names.CONTENT_LENGTH, finalResponse.content().readableBytes()); + } + } else { + finalResponse = httpResponse; + } + + finalResponse.headers().set(HttpHeaders.Names.ACCESS_CONTROL_ALLOW_ORIGIN, allowOrigin); + KeepAliveWrite.flush(ctx, routed.request(), finalResponse); + }); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java index 17f02f0dcaab9..e74541ec889ad 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/WebRuntimeMonitor.java @@ -232,41 +232,41 @@ public WebRuntimeMonitor( Router router = new Router(); // config how to interact with this web server - get(router, new DashboardConfigHandler(cfg.getRefreshInterval())); + get(router, new DashboardConfigHandler(executor, cfg.getRefreshInterval())); // the overview - how many task managers, slots, free slots, ... - get(router, new ClusterOverviewHandler(DEFAULT_REQUEST_TIMEOUT)); + get(router, new ClusterOverviewHandler(executor, DEFAULT_REQUEST_TIMEOUT)); // job manager configuration - get(router, new JobManagerConfigHandler(config)); + get(router, new JobManagerConfigHandler(executor, config)); // overview over jobs - get(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, true)); - get(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, true, false)); - get(router, new CurrentJobsOverviewHandler(DEFAULT_REQUEST_TIMEOUT, false, true)); - - get(router, new CurrentJobIdsHandler(DEFAULT_REQUEST_TIMEOUT)); - - get(router, new JobDetailsHandler(currentGraphs, metricFetcher)); - - get(router, new JobVertexDetailsHandler(currentGraphs, metricFetcher)); - get(router, new SubtasksTimesHandler(currentGraphs)); - get(router, new JobVertexTaskManagersHandler(currentGraphs, metricFetcher)); - get(router, new JobVertexAccumulatorsHandler(currentGraphs)); - get(router, new JobVertexBackPressureHandler(currentGraphs, backPressureStatsTracker, refreshInterval)); - get(router, new JobVertexMetricsHandler(metricFetcher)); - get(router, new SubtasksAllAccumulatorsHandler(currentGraphs)); - get(router, new SubtaskCurrentAttemptDetailsHandler(currentGraphs, metricFetcher)); - get(router, new SubtaskExecutionAttemptDetailsHandler(currentGraphs, metricFetcher)); - get(router, new SubtaskExecutionAttemptAccumulatorsHandler(currentGraphs)); - - get(router, new JobPlanHandler(currentGraphs)); - get(router, new JobConfigHandler(currentGraphs)); - get(router, new JobExceptionsHandler(currentGraphs)); - get(router, new JobAccumulatorsHandler(currentGraphs)); - get(router, new JobMetricsHandler(metricFetcher)); - - get(router, new TaskManagersHandler(DEFAULT_REQUEST_TIMEOUT, metricFetcher)); + get(router, new CurrentJobsOverviewHandler(executor, DEFAULT_REQUEST_TIMEOUT, true, true)); + get(router, new CurrentJobsOverviewHandler(executor, DEFAULT_REQUEST_TIMEOUT, true, false)); + get(router, new CurrentJobsOverviewHandler(executor, DEFAULT_REQUEST_TIMEOUT, false, true)); + + get(router, new CurrentJobIdsHandler(executor, DEFAULT_REQUEST_TIMEOUT)); + + get(router, new JobDetailsHandler(currentGraphs, executor, metricFetcher)); + + get(router, new JobVertexDetailsHandler(currentGraphs, executor, metricFetcher)); + get(router, new SubtasksTimesHandler(currentGraphs, executor)); + get(router, new JobVertexTaskManagersHandler(currentGraphs, executor, metricFetcher)); + get(router, new JobVertexAccumulatorsHandler(currentGraphs, executor)); + get(router, new JobVertexBackPressureHandler(currentGraphs, executor, backPressureStatsTracker, refreshInterval)); + get(router, new JobVertexMetricsHandler(executor, metricFetcher)); + get(router, new SubtasksAllAccumulatorsHandler(currentGraphs, executor)); + get(router, new SubtaskCurrentAttemptDetailsHandler(currentGraphs, executor, metricFetcher)); + get(router, new SubtaskExecutionAttemptDetailsHandler(currentGraphs, executor, metricFetcher)); + get(router, new SubtaskExecutionAttemptAccumulatorsHandler(currentGraphs, executor)); + + get(router, new JobPlanHandler(currentGraphs, executor)); + get(router, new JobConfigHandler(currentGraphs, executor)); + get(router, new JobExceptionsHandler(currentGraphs, executor)); + get(router, new JobAccumulatorsHandler(currentGraphs, executor)); + get(router, new JobMetricsHandler(executor, metricFetcher)); + + get(router, new TaskManagersHandler(executor, DEFAULT_REQUEST_TIMEOUT, metricFetcher)); get(router, new TaskManagerLogHandler( retriever, @@ -287,7 +287,7 @@ public WebRuntimeMonitor( config, enableSSL, blobView)); - get(router, new TaskManagerMetricsHandler(metricFetcher)); + get(router, new TaskManagerMetricsHandler(executor, metricFetcher)); router // log and stdout @@ -299,48 +299,48 @@ public WebRuntimeMonitor( new StaticFileServerHandler(retriever, jobManagerAddressFuture, timeout, logFiles.stdOutFile, enableSSL)); - get(router, new JobManagerMetricsHandler(metricFetcher)); + get(router, new JobManagerMetricsHandler(executor, metricFetcher)); // Cancel a job via GET (for proper integration with YARN this has to be performed via GET) - get(router, new JobCancellationHandler(timeout)); + get(router, new JobCancellationHandler(executor, timeout)); // DELETE is the preferred way of canceling a job (Rest-conform) - delete(router, new JobCancellationHandler(timeout)); + delete(router, new JobCancellationHandler(executor, timeout)); get(router, triggerHandler); get(router, inProgressHandler); // stop a job via GET (for proper integration with YARN this has to be performed via GET) - get(router, new JobStoppingHandler(timeout)); + get(router, new JobStoppingHandler(executor, timeout)); // DELETE is the preferred way of stopping a job (Rest-conform) - delete(router, new JobStoppingHandler(timeout)); + delete(router, new JobStoppingHandler(executor, timeout)); int maxCachedEntries = config.getInteger(WebOptions.CHECKPOINTS_HISTORY_SIZE); CheckpointStatsCache cache = new CheckpointStatsCache(maxCachedEntries); // Register the checkpoint stats handlers - get(router, new CheckpointStatsHandler(currentGraphs)); - get(router, new CheckpointConfigHandler(currentGraphs)); - get(router, new CheckpointStatsDetailsHandler(currentGraphs, cache)); - get(router, new CheckpointStatsDetailsSubtasksHandler(currentGraphs, cache)); + get(router, new CheckpointStatsHandler(currentGraphs, executor)); + get(router, new CheckpointConfigHandler(currentGraphs, executor)); + get(router, new CheckpointStatsDetailsHandler(currentGraphs, executor, cache)); + get(router, new CheckpointStatsDetailsSubtasksHandler(currentGraphs, executor, cache)); if (webSubmitAllow) { // fetch the list of uploaded jars. - get(router, new JarListHandler(uploadDir)); + get(router, new JarListHandler(executor, uploadDir)); // get plan for an uploaded jar - get(router, new JarPlanHandler(uploadDir)); + get(router, new JarPlanHandler(executor, uploadDir)); // run a jar - post(router, new JarRunHandler(uploadDir, timeout, config)); + post(router, new JarRunHandler(executor, uploadDir, timeout, config)); // upload a jar - post(router, new JarUploadHandler(uploadDir)); + post(router, new JarUploadHandler(executor, uploadDir)); // delete an uploaded jar from submission interface - delete(router, new JarDeleteHandler(uploadDir)); + delete(router, new JarDeleteHandler(executor, uploadDir)); } else { // send an Access Denied message - JarAccessDeniedHandler jad = new JarAccessDeniedHandler(); + JarAccessDeniedHandler jad = new JarAccessDeniedHandler(executor); get(router, jad); post(router, jad); delete(router, jad); diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java index 89108db5f4dab..053d3f7b69a81 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractExecutionGraphRequestHandler.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.concurrent.FlinkFutureException; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; @@ -28,6 +30,8 @@ import java.util.Map; import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Base class for request handlers whose response depends on an ExecutionGraph @@ -37,12 +41,16 @@ public abstract class AbstractExecutionGraphRequestHandler extends AbstractJsonR private final ExecutionGraphHolder executionGraphHolder; - public AbstractExecutionGraphRequestHandler(ExecutionGraphHolder executionGraphHolder) { + public AbstractExecutionGraphRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executor); this.executionGraphHolder = Preconditions.checkNotNull(executionGraphHolder); } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { + public CompletableFuture handleJsonRequest( + Map pathParams, + Map queryParams, + JobManagerGateway jobManagerGateway) { String jidString = pathParams.get("jobid"); if (jidString == null) { throw new RuntimeException("JobId parameter missing"); @@ -53,21 +61,20 @@ public String handleJsonRequest(Map pathParams, Map optGraph; + final CompletableFuture> graphFuture = executionGraphHolder.getExecutionGraph(jid, jobManagerGateway); - try { - optGraph = executionGraphHolder.getExecutionGraph(jid, jobManagerGateway); - } catch (Exception e) { - throw new FlinkException("Could not retrieve ExecutionGraph for job with jobId " + jid + " from the JobManager.", e); - } - - final AccessExecutionGraph graph = optGraph.orElseThrow(() -> new NotFoundException("Could not find job with jobId " + jid + '.')); - - return handleRequest(graph, pathParams); + return graphFuture.thenComposeAsync( + (Optional optGraph) -> { + if (optGraph.isPresent()) { + return handleRequest(optGraph.get(), pathParams); + } else { + throw new FlinkFutureException(new NotFoundException("Could not find job with jobId " + jid + '.')); + } + }, executor); } - public abstract String handleRequest(AccessExecutionGraph graph, Map params) throws Exception; + public abstract CompletableFuture handleRequest(AccessExecutionGraph graph, Map params); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJobVertexRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJobVertexRequestHandler.java index af9fc6c4fd331..df09225a384fa 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJobVertexRequestHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJobVertexRequestHandler.java @@ -24,6 +24,8 @@ import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Base class for request handlers whose response depends on a specific job vertex (defined @@ -31,12 +33,12 @@ */ public abstract class AbstractJobVertexRequestHandler extends AbstractExecutionGraphRequestHandler { - public AbstractJobVertexRequestHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public AbstractJobVertexRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override - public final String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { + public final CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { final JobVertexID vid = parseJobVertexId(params); final AccessExecutionJobVertex jobVertex = graph.getJobVertex(vid); @@ -66,5 +68,5 @@ public static JobVertexID parseJobVertexId(Map params) { } } - public abstract String handleRequest(AccessExecutionJobVertex jobVertex, Map params) throws Exception; + public abstract CompletableFuture handleRequest(AccessExecutionJobVertex jobVertex, Map params); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java index 266ffb055c4c1..1ec3f9cca002b 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractJsonRequestHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.runtime.jobmaster.JobManagerGateway; +import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse; @@ -29,6 +30,8 @@ import java.nio.charset.Charset; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Base class for most request handlers. The handlers must produce a JSON response. @@ -37,18 +40,28 @@ public abstract class AbstractJsonRequestHandler implements RequestHandler { private static final Charset ENCODING = Charset.forName("UTF-8"); + protected final Executor executor; + + protected AbstractJsonRequestHandler(Executor executor) { + this.executor = Preconditions.checkNotNull(executor); + } + @Override - public FullHttpResponse handleRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - String result = handleJsonRequest(pathParams, queryParams, jobManagerGateway); - byte[] bytes = result.getBytes(ENCODING); + public CompletableFuture handleRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + CompletableFuture resultFuture = handleJsonRequest(pathParams, queryParams, jobManagerGateway); + + return resultFuture.thenApplyAsync( + (String result) -> { + byte[] bytes = result.getBytes(ENCODING); - DefaultFullHttpResponse response = new DefaultFullHttpResponse( - HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(bytes)); + DefaultFullHttpResponse response = new DefaultFullHttpResponse( + HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(bytes)); - response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name()); - response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes()); + response.headers().set(HttpHeaders.Names.CONTENT_TYPE, "application/json; charset=" + ENCODING.name()); + response.headers().set(HttpHeaders.Names.CONTENT_LENGTH, response.content().readableBytes()); - return response; + return response; + }); } /** @@ -66,9 +79,9 @@ public FullHttpResponse handleRequest(Map pathParams, Map handleJsonRequest( Map pathParams, Map queryParams, - JobManagerGateway jobManagerGateway) throws Exception; + JobManagerGateway jobManagerGateway); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskAttemptRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskAttemptRequestHandler.java index 27920084c8844..1b20673583c40 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskAttemptRequestHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskAttemptRequestHandler.java @@ -18,11 +18,15 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecution; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; +import org.apache.flink.util.FlinkException; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Base class for request handlers whose response depends on a specific subtask execution attempt @@ -32,15 +36,15 @@ */ public abstract class AbstractSubtaskAttemptRequestHandler extends AbstractSubtaskRequestHandler { - public AbstractSubtaskAttemptRequestHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public AbstractSubtaskAttemptRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override - public String handleRequest(AccessExecutionVertex vertex, Map params) throws Exception { + public CompletableFuture handleRequest(AccessExecutionVertex vertex, Map params) { final String attemptNumberString = params.get("attempt"); if (attemptNumberString == null) { - throw new RuntimeException("Attempt number parameter missing"); + return FutureUtils.completedExceptionally(new FlinkException("Attempt number parameter missing")); } final int attempt; @@ -48,7 +52,7 @@ public String handleRequest(AccessExecutionVertex vertex, Map pa attempt = Integer.parseInt(attemptNumberString); } catch (NumberFormatException e) { - throw new RuntimeException("Invalid attempt number parameter"); + return FutureUtils.completedExceptionally(new FlinkException("Invalid attempt number parameter")); } final AccessExecution currentAttempt = vertex.getCurrentExecutionAttempt(); @@ -61,14 +65,14 @@ else if (attempt >= 0 && attempt < currentAttempt.getAttemptNumber()) { if (exec != null) { return handleRequest(exec, params); } else { - throw new RequestHandlerException("Execution for attempt " + attempt + - " has already been deleted."); + return FutureUtils.completedExceptionally(new RequestHandlerException("Execution for attempt " + attempt + + " has already been deleted.")); } } else { - throw new RuntimeException("Attempt does not exist: " + attempt); + return FutureUtils.completedExceptionally(new FlinkException("Attempt does not exist: " + attempt)); } } - public abstract String handleRequest(AccessExecution execAttempt, Map params) throws Exception; + public abstract CompletableFuture handleRequest(AccessExecution execAttempt, Map params); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskRequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskRequestHandler.java index b9772285fe20b..ab850349899a9 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskRequestHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/AbstractSubtaskRequestHandler.java @@ -18,11 +18,15 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; +import org.apache.flink.util.FlinkException; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Base class for request handlers whose response depends on a specific subtask (defined via the @@ -31,15 +35,15 @@ */ public abstract class AbstractSubtaskRequestHandler extends AbstractJobVertexRequestHandler { - public AbstractSubtaskRequestHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public AbstractSubtaskRequestHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override - public final String handleRequest(AccessExecutionJobVertex jobVertex, Map params) throws Exception { + public final CompletableFuture handleRequest(AccessExecutionJobVertex jobVertex, Map params) { final String subtaskNumberString = params.get("subtasknum"); if (subtaskNumberString == null) { - throw new RuntimeException("Subtask number parameter missing"); + return FutureUtils.completedExceptionally(new FlinkException("Subtask number parameter missing")); } final int subtask; @@ -47,16 +51,16 @@ public final String handleRequest(AccessExecutionJobVertex jobVertex, Map= jobVertex.getParallelism()) { - throw new RuntimeException("subtask does not exist: " + subtask); + return FutureUtils.completedExceptionally(new FlinkException("subtask does not exist: " + subtask)); } final AccessExecutionVertex vertex = jobVertex.getTaskVertices()[subtask]; return handleRequest(vertex, params); } - public abstract String handleRequest(AccessExecutionVertex vertex, Map params) throws Exception; + public abstract CompletableFuture handleRequest(AccessExecutionVertex vertex, Map params); } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java index 4ebc4e73b76b5..17db2e81f9483 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandler.java @@ -19,16 +19,20 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FlinkFutureException; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.messages.webmonitor.StatusOverview; import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.util.FlinkException; import com.fasterxml.jackson.core.JsonGenerator; +import java.io.IOException; import java.io.StringWriter; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.Executor; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -46,7 +50,8 @@ public class ClusterOverviewHandler extends AbstractJsonRequestHandler { private final Time timeout; - public ClusterOverviewHandler(Time timeout) { + public ClusterOverviewHandler(Executor executor, Time timeout) { + super(executor); this.timeout = checkNotNull(timeout); } @@ -56,39 +61,45 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { // we need no parameters, get all requests try { if (jobManagerGateway != null) { CompletableFuture overviewFuture = jobManagerGateway.requestStatusOverview(timeout); - StatusOverview overview = overviewFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); - - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - - gen.writeStartObject(); - gen.writeNumberField("taskmanagers", overview.getNumTaskManagersConnected()); - gen.writeNumberField("slots-total", overview.getNumSlotsTotal()); - gen.writeNumberField("slots-available", overview.getNumSlotsAvailable()); - gen.writeNumberField("jobs-running", overview.getNumJobsRunningOrPending()); - gen.writeNumberField("jobs-finished", overview.getNumJobsFinished()); - gen.writeNumberField("jobs-cancelled", overview.getNumJobsCancelled()); - gen.writeNumberField("jobs-failed", overview.getNumJobsFailed()); - gen.writeStringField("flink-version", version); - if (!commitID.equals(EnvironmentInformation.UNKNOWN)) { - gen.writeStringField("flink-commit", commitID); - } - gen.writeEndObject(); - - gen.close(); - return writer.toString(); + return overviewFuture.thenApplyAsync( + (StatusOverview overview) -> { + StringWriter writer = new StringWriter(); + try { + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + + gen.writeStartObject(); + gen.writeNumberField("taskmanagers", overview.getNumTaskManagersConnected()); + gen.writeNumberField("slots-total", overview.getNumSlotsTotal()); + gen.writeNumberField("slots-available", overview.getNumSlotsAvailable()); + gen.writeNumberField("jobs-running", overview.getNumJobsRunningOrPending()); + gen.writeNumberField("jobs-finished", overview.getNumJobsFinished()); + gen.writeNumberField("jobs-cancelled", overview.getNumJobsCancelled()); + gen.writeNumberField("jobs-failed", overview.getNumJobsFailed()); + gen.writeStringField("flink-version", version); + if (!commitID.equals(EnvironmentInformation.UNKNOWN)) { + gen.writeStringField("flink-commit", commitID); + } + gen.writeEndObject(); + + gen.close(); + return writer.toString(); + } catch (IOException exception) { + throw new FlinkFutureException("Could not write cluster overview.", exception); + } + }, + executor); } else { throw new Exception("No connection to the leading JobManager."); } } catch (Exception e) { - throw new RuntimeException("Failed to fetch list of all running jobs: " + e.getMessage(), e); + return FutureUtils.completedExceptionally(new FlinkException("Failed to fetch list of all running jobs: ", e)); } } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java index 778a300dbac65..acf1cd009bc61 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandler.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.messages.webmonitor.JobsWithIDsOverview; @@ -28,6 +29,7 @@ import java.io.StringWriter; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import static java.util.Objects.requireNonNull; @@ -43,7 +45,8 @@ public class CurrentJobIdsHandler extends AbstractJsonRequestHandler { private final Time timeout; - public CurrentJobIdsHandler(Time timeout) { + public CurrentJobIdsHandler(Executor executor, Time timeout) { + super(executor); this.timeout = requireNonNull(timeout); } @@ -53,53 +56,57 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - // we need no parameters, get all requests - try { - if (jobManagerGateway != null) { - CompletableFuture overviewFuture = jobManagerGateway.requestJobsOverview(timeout); - JobsWithIDsOverview overview = overviewFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); - - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - - gen.writeStartObject(); - - gen.writeArrayFieldStart("jobs-running"); - for (JobID jid : overview.getJobsRunningOrPending()) { - gen.writeString(jid.toString()); - } - gen.writeEndArray(); - - gen.writeArrayFieldStart("jobs-finished"); - for (JobID jid : overview.getJobsFinished()) { - gen.writeString(jid.toString()); - } - gen.writeEndArray(); - - gen.writeArrayFieldStart("jobs-cancelled"); - for (JobID jid : overview.getJobsCancelled()) { - gen.writeString(jid.toString()); + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.supplyAsync( + () -> { + // we need no parameters, get all requests + try { + if (jobManagerGateway != null) { + CompletableFuture overviewFuture = jobManagerGateway.requestJobsOverview(timeout); + JobsWithIDsOverview overview = overviewFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + StringWriter writer = new StringWriter(); + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + + gen.writeStartObject(); + + gen.writeArrayFieldStart("jobs-running"); + for (JobID jid : overview.getJobsRunningOrPending()) { + gen.writeString(jid.toString()); + } + gen.writeEndArray(); + + gen.writeArrayFieldStart("jobs-finished"); + for (JobID jid : overview.getJobsFinished()) { + gen.writeString(jid.toString()); + } + gen.writeEndArray(); + + gen.writeArrayFieldStart("jobs-cancelled"); + for (JobID jid : overview.getJobsCancelled()) { + gen.writeString(jid.toString()); + } + gen.writeEndArray(); + + gen.writeArrayFieldStart("jobs-failed"); + for (JobID jid : overview.getJobsFailed()) { + gen.writeString(jid.toString()); + } + gen.writeEndArray(); + + gen.writeEndObject(); + + gen.close(); + return writer.toString(); + } + else { + throw new Exception("No connection to the leading JobManager."); + } } - gen.writeEndArray(); - - gen.writeArrayFieldStart("jobs-failed"); - for (JobID jid : overview.getJobsFailed()) { - gen.writeString(jid.toString()); + catch (Exception e) { + throw new FlinkFutureException("Failed to fetch list of all running jobs.", e); } - gen.writeEndArray(); - - gen.writeEndObject(); - - gen.close(); - return writer.toString(); - } - else { - throw new Exception("No connection to the leading JobManager."); - } - } - catch (Exception e) { - throw new RuntimeException("Failed to fetch list of all running jobs: " + e.getMessage(), e); - } + }, + executor); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java index b3244267ee09a..a5b116c585139 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandler.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FlinkFutureException; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobmaster.JobManagerGateway; @@ -36,7 +38,7 @@ import java.util.Collections; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.Executor; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -55,10 +57,12 @@ public class CurrentJobsOverviewHandler extends AbstractJsonRequestHandler { private final boolean includeFinishedJobs; public CurrentJobsOverviewHandler( + Executor executor, Time timeout, boolean includeRunningJobs, boolean includeFinishedJobs) { + super(executor); this.timeout = checkNotNull(timeout); this.includeRunningJobs = includeRunningJobs; this.includeFinishedJobs = includeFinishedJobs; @@ -77,49 +81,50 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - try { - if (jobManagerGateway != null) { - CompletableFuture jobDetailsFuture = jobManagerGateway.requestJobDetails(includeRunningJobs, includeFinishedJobs, timeout); - MultipleJobsDetails result = jobDetailsFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); - - final long now = System.currentTimeMillis(); - - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - gen.writeStartObject(); - - if (includeRunningJobs && includeFinishedJobs) { - gen.writeArrayFieldStart("running"); - for (JobDetails detail : result.getRunningJobs()) { - writeJobDetailOverviewAsJson(detail, gen, now); - } - gen.writeEndArray(); - - gen.writeArrayFieldStart("finished"); - for (JobDetails detail : result.getFinishedJobs()) { - writeJobDetailOverviewAsJson(detail, gen, now); + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + if (jobManagerGateway != null) { + CompletableFuture jobDetailsFuture = jobManagerGateway.requestJobDetails(includeRunningJobs, includeFinishedJobs, timeout); + + return jobDetailsFuture.thenApplyAsync( + (MultipleJobsDetails result) -> { + final long now = System.currentTimeMillis(); + + StringWriter writer = new StringWriter(); + try { + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + gen.writeStartObject(); + + if (includeRunningJobs && includeFinishedJobs) { + gen.writeArrayFieldStart("running"); + for (JobDetails detail : result.getRunningJobs()) { + writeJobDetailOverviewAsJson(detail, gen, now); + } + gen.writeEndArray(); + + gen.writeArrayFieldStart("finished"); + for (JobDetails detail : result.getFinishedJobs()) { + writeJobDetailOverviewAsJson(detail, gen, now); + } + gen.writeEndArray(); + } else { + gen.writeArrayFieldStart("jobs"); + for (JobDetails detail : includeRunningJobs ? result.getRunningJobs() : result.getFinishedJobs()) { + writeJobDetailOverviewAsJson(detail, gen, now); + } + gen.writeEndArray(); + } + + gen.writeEndObject(); + gen.close(); + return writer.toString(); + } catch (IOException e) { + throw new FlinkFutureException("Could not write current jobs overview json.", e); } - gen.writeEndArray(); - } - else { - gen.writeArrayFieldStart("jobs"); - for (JobDetails detail : includeRunningJobs ? result.getRunningJobs() : result.getFinishedJobs()) { - writeJobDetailOverviewAsJson(detail, gen, now); - } - gen.writeEndArray(); - } - - gen.writeEndObject(); - gen.close(); - return writer.toString(); - } - else { - throw new Exception("No connection to the leading JobManager."); - } + }, + executor); } - catch (Exception e) { - throw new Exception("Failed to fetch the status overview: " + e.getMessage(), e); + else { + return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager.")); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java index fe1d06bc3d713..39984b173f713 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandler.java @@ -27,6 +27,8 @@ import java.io.StringWriter; import java.util.Map; import java.util.TimeZone; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Responder that returns the parameters that define how the asynchronous requests @@ -39,7 +41,8 @@ public class DashboardConfigHandler extends AbstractJsonRequestHandler { private final String configString; - public DashboardConfigHandler(long refreshInterval) { + public DashboardConfigHandler(Executor executor, long refreshInterval) { + super(executor); try { this.configString = createConfigJson(refreshInterval); } @@ -55,8 +58,8 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - return this.configString; + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.completedFuture(configString); } public static String createConfigJson(long refreshInterval) throws IOException { diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java index db55169f55829..978432bddfb2f 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandler.java @@ -21,6 +21,8 @@ import org.apache.flink.runtime.jobmaster.JobManagerGateway; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Handler to deny access to jar-related REST calls. @@ -30,6 +32,10 @@ public class JarAccessDeniedHandler extends AbstractJsonRequestHandler { private static final String ERROR_MESSAGE = "{\"error\": \"Web submission interface is not " + "available for this cluster. To enable it, set the configuration key ' jobmanager.web.submit.enable.'\"}"; + public JarAccessDeniedHandler(Executor executor) { + super(executor); + } + @Override public String[] getPaths() { return new String[]{ @@ -42,7 +48,7 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - return ERROR_MESSAGE; + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.completedFuture(ERROR_MESSAGE); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java index d86a21bec4a49..0b0d32e7b9272 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarActionHandler.java @@ -46,6 +46,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.concurrent.Executor; /** * Abstract handler for fetching plan for a jar or running a jar. @@ -54,7 +55,8 @@ public abstract class JarActionHandler extends AbstractJsonRequestHandler { private final File jarDir; - public JarActionHandler(File jarDirectory) { + public JarActionHandler(Executor executor, File jarDirectory) { + super(executor); jarDir = jarDirectory; } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java index 73771bdc2fc6e..d9df1d49f64e1 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import com.fasterxml.jackson.core.JsonGenerator; @@ -26,6 +27,8 @@ import java.io.FilenameFilter; import java.io.StringWriter; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Handles requests for deletion of jars. @@ -36,7 +39,8 @@ public class JarDeleteHandler extends AbstractJsonRequestHandler { private final File jarDir; - public JarDeleteHandler(File jarDirectory) { + public JarDeleteHandler(Executor executor, File jarDirectory) { + super(executor); jarDir = jarDirectory; } @@ -46,33 +50,37 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { final String file = pathParams.get("jarid"); - try { - File[] list = jarDir.listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - return name.equals(file); + return CompletableFuture.supplyAsync( + () -> { + try { + File[] list = jarDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.equals(file); + } + }); + boolean success = false; + for (File f: list) { + // although next to impossible for multiple files, we still delete them. + success = success || f.delete(); + } + StringWriter writer = new StringWriter(); + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + gen.writeStartObject(); + if (!success) { + // this seems to always fail on Windows. + gen.writeStringField("error", "The requested jar couldn't be deleted. Please try again."); + } + gen.writeEndObject(); + gen.close(); + return writer.toString(); } - }); - boolean success = false; - for (File f: list) { - // although next to impossible for multiple files, we still delete them. - success = success || f.delete(); - } - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - gen.writeStartObject(); - if (!success) { - // this seems to always fail on Windows. - gen.writeStringField("error", "The requested jar couldn't be deleted. Please try again."); - } - gen.writeEndObject(); - gen.close(); - return writer.toString(); - } - catch (Exception e) { - throw new RuntimeException("Failed to delete jar id " + pathParams.get("jarid") + ": " + e.getMessage(), e); - } + catch (Exception e) { + throw new FlinkFutureException("Failed to delete jar id " + pathParams.get("jarid") + '.', e); + } + }, + executor); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java index 4f9b188f64639..95281a48756af 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.webmonitor.RuntimeMonitorHandler; @@ -29,6 +30,8 @@ import java.io.IOException; import java.io.StringWriter; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import java.util.jar.JarFile; import java.util.jar.Manifest; @@ -41,7 +44,8 @@ public class JarListHandler extends AbstractJsonRequestHandler { private final File jarDir; - public JarListHandler(File jarDirectory) { + public JarListHandler(Executor executor, File jarDirectory) { + super(executor); jarDir = jarDirectory; } @@ -51,88 +55,93 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - try { - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - - gen.writeStartObject(); - gen.writeStringField("address", queryParams.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY)); - gen.writeArrayFieldStart("files"); - - File[] list = jarDir.listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - return name.endsWith(".jar"); - } - }); + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.supplyAsync( + () -> { + try { + StringWriter writer = new StringWriter(); + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - for (File f : list) { - // separate the uuid and the name parts. - String id = f.getName(); + gen.writeStartObject(); + gen.writeStringField("address", queryParams.get(RuntimeMonitorHandler.WEB_MONITOR_ADDRESS_KEY)); + gen.writeArrayFieldStart("files"); - int startIndex = id.indexOf("_"); - if (startIndex < 0) { - continue; - } - String name = id.substring(startIndex + 1); - if (name.length() < 5 || !name.endsWith(".jar")) { - continue; - } + File[] list = jarDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.endsWith(".jar"); + } + }); - gen.writeStartObject(); - gen.writeStringField("id", id); - gen.writeStringField("name", name); - gen.writeNumberField("uploaded", f.lastModified()); - gen.writeArrayFieldStart("entry"); + for (File f : list) { + // separate the uuid and the name parts. + String id = f.getName(); - String[] classes = new String[0]; - try { - JarFile jar = new JarFile(f); - Manifest manifest = jar.getManifest(); - String assemblerClass = null; - - if (manifest != null) { - assemblerClass = manifest.getMainAttributes().getValue(PackagedProgram.MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS); - if (assemblerClass == null) { - assemblerClass = manifest.getMainAttributes().getValue(PackagedProgram.MANIFEST_ATTRIBUTE_MAIN_CLASS); + int startIndex = id.indexOf("_"); + if (startIndex < 0) { + continue; + } + String name = id.substring(startIndex + 1); + if (name.length() < 5 || !name.endsWith(".jar")) { + continue; } - } - if (assemblerClass != null) { - classes = assemblerClass.split(","); - } - } catch (IOException ignored) { - // we simply show no entries here - } - - // show every entry class that can be loaded later on. - for (String clazz : classes) { - clazz = clazz.trim(); - PackagedProgram program = null; - try { - program = new PackagedProgram(f, clazz, new String[0]); - } catch (Exception ignored) { - // ignore jar files which throw an error upon creating a PackagedProgram - } - if (program != null) { gen.writeStartObject(); - gen.writeStringField("name", clazz); - String desc = program.getDescription(); - gen.writeStringField("description", desc == null ? "No description provided" : desc); + gen.writeStringField("id", id); + gen.writeStringField("name", name); + gen.writeNumberField("uploaded", f.lastModified()); + gen.writeArrayFieldStart("entry"); + + String[] classes = new String[0]; + try { + JarFile jar = new JarFile(f); + Manifest manifest = jar.getManifest(); + String assemblerClass = null; + + if (manifest != null) { + assemblerClass = manifest.getMainAttributes().getValue(PackagedProgram.MANIFEST_ATTRIBUTE_ASSEMBLER_CLASS); + if (assemblerClass == null) { + assemblerClass = manifest.getMainAttributes().getValue(PackagedProgram.MANIFEST_ATTRIBUTE_MAIN_CLASS); + } + } + if (assemblerClass != null) { + classes = assemblerClass.split(","); + } + } catch (IOException ignored) { + // we simply show no entries here + } + + // show every entry class that can be loaded later on. + for (String clazz : classes) { + clazz = clazz.trim(); + + PackagedProgram program = null; + try { + program = new PackagedProgram(f, clazz, new String[0]); + } catch (Exception ignored) { + // ignore jar files which throw an error upon creating a PackagedProgram + } + if (program != null) { + gen.writeStartObject(); + gen.writeStringField("name", clazz); + String desc = program.getDescription(); + gen.writeStringField("description", desc == null ? "No description provided" : desc); + gen.writeEndObject(); + } + } + gen.writeEndArray(); gen.writeEndObject(); } + gen.writeEndArray(); + gen.writeEndObject(); + gen.close(); + return writer.toString(); } - gen.writeEndArray(); - gen.writeEndObject(); - } - gen.writeEndArray(); - gen.writeEndObject(); - gen.close(); - return writer.toString(); - } - catch (Exception e) { - throw new RuntimeException("Failed to fetch jar list: " + e.getMessage(), e); - } + catch (Exception e) { + throw new FlinkFutureException("Failed to fetch jar list.", e); + } + }, + executor); + } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java index b239160f2b1e4..b117b3d71df5a 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator; import org.apache.flink.runtime.jobmaster.JobManagerGateway; @@ -27,6 +28,8 @@ import java.io.File; import java.io.StringWriter; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * This handler handles requests to fetch plan for a jar. @@ -35,8 +38,8 @@ public class JarPlanHandler extends JarActionHandler { static final String JAR_PLAN_REST_PATH = "/jars/:jarid/plan"; - public JarPlanHandler(File jarDirectory) { - super(jarDirectory); + public JarPlanHandler(Executor executor, File jarDirectory) { + super(executor, jarDirectory); } @Override @@ -45,21 +48,25 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - try { - JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams); - JobGraph graph = getJobGraphAndClassLoader(config).f0; - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - gen.writeStartObject(); - gen.writeFieldName("plan"); - gen.writeRawValue(JsonPlanGenerator.generatePlan(graph)); - gen.writeEndObject(); - gen.close(); - return writer.toString(); - } - catch (Exception e) { - return sendError(e); - } + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.supplyAsync( + () -> { + try { + JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams); + JobGraph graph = getJobGraphAndClassLoader(config).f0; + StringWriter writer = new StringWriter(); + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + gen.writeStartObject(); + gen.writeFieldName("plan"); + gen.writeRawValue(JsonPlanGenerator.generatePlan(graph)); + gen.writeEndObject(); + gen.close(); + return writer.toString(); + } + catch (Exception e) { + throw new FlinkFutureException(e); + } + }, + executor); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java index 12ffa4fb2ac01..7ada0b4e1a605 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandler.java @@ -24,6 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.client.JobClient; import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.util.Preconditions; @@ -33,6 +34,8 @@ import java.io.File; import java.io.StringWriter; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * This handler handles requests to fetch plan for a jar. @@ -44,8 +47,8 @@ public class JarRunHandler extends JarActionHandler { private final Time timeout; private final Configuration clientConfig; - public JarRunHandler(File jarDirectory, Time timeout, Configuration clientConfig) { - super(jarDirectory); + public JarRunHandler(Executor executor, File jarDirectory, Time timeout, Configuration clientConfig) { + super(executor, jarDirectory); this.timeout = Preconditions.checkNotNull(timeout); this.clientConfig = Preconditions.checkNotNull(clientConfig); } @@ -56,31 +59,35 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - try { - JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams); - Tuple2 graph = getJobGraphAndClassLoader(config); + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.supplyAsync( + () -> { + try { + JarActionHandlerConfig config = JarActionHandlerConfig.fromParams(pathParams, queryParams); + Tuple2 graph = getJobGraphAndClassLoader(config); - try { - JobClient.submitJobDetached( - jobManagerGateway, - clientConfig, - graph.f0, - timeout, - graph.f1); - } catch (JobExecutionException e) { - throw new ProgramInvocationException("Failed to submit the job to the job manager", e); - } + try { + JobClient.submitJobDetached( + jobManagerGateway, + clientConfig, + graph.f0, + timeout, + graph.f1); + } catch (JobExecutionException e) { + throw new ProgramInvocationException("Failed to submit the job to the job manager", e); + } - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - gen.writeStartObject(); - gen.writeStringField("jobid", graph.f0.getJobID().toString()); - gen.writeEndObject(); - gen.close(); - return writer.toString(); - } catch (Exception e) { - return sendError(e); - } + StringWriter writer = new StringWriter(); + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + gen.writeStartObject(); + gen.writeStringField("jobid", graph.f0.getJobID().toString()); + gen.writeEndObject(); + gen.close(); + return writer.toString(); + } catch (Exception e) { + throw new FlinkFutureException("Could not run the jar.", e); + } + }, + executor); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java index 705c321aba8d5..61b3f585aca2c 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandler.java @@ -23,6 +23,8 @@ import java.io.File; import java.util.Map; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Handles requests for uploading of jars. @@ -33,7 +35,8 @@ public class JarUploadHandler extends AbstractJsonRequestHandler { private final File jarDir; - public JarUploadHandler(File jarDir) { + public JarUploadHandler(Executor executor, File jarDir) { + super(executor); this.jarDir = jarDir; } @@ -43,34 +46,38 @@ public String[] getPaths() { } @Override - public String handleJsonRequest( + public CompletableFuture handleJsonRequest( Map pathParams, Map queryParams, - JobManagerGateway jobManagerGateway) throws Exception { + JobManagerGateway jobManagerGateway) { String tempFilePath = queryParams.get("filepath"); String filename = queryParams.get("filename"); - File tempFile; - if (tempFilePath != null && (tempFile = new File(tempFilePath)).exists()) { - if (!tempFile.getName().endsWith(".jar")) { - //noinspection ResultOfMethodCallIgnored - tempFile.delete(); - return "{\"error\": \"Only Jar files are allowed.\"}"; - } + return CompletableFuture.supplyAsync( + () -> { + File tempFile; + if (tempFilePath != null && (tempFile = new File(tempFilePath)).exists()) { + if (!tempFile.getName().endsWith(".jar")) { + //noinspection ResultOfMethodCallIgnored + tempFile.delete(); + return "{\"error\": \"Only Jar files are allowed.\"}"; + } - String filenameWithUUID = UUID.randomUUID() + "_" + filename; - File newFile = new File(jarDir, filenameWithUUID); - if (tempFile.renameTo(newFile)) { - // all went well - return "{\"status\": \"success\", \"filename\": \"" + filenameWithUUID + "\"}"; - } - else { - //noinspection ResultOfMethodCallIgnored - tempFile.delete(); - } - } + String filenameWithUUID = UUID.randomUUID() + "_" + filename; + File newFile = new File(jarDir, filenameWithUUID); + if (tempFile.renameTo(newFile)) { + // all went well + return "{\"status\": \"success\", \"filename\": \"" + filenameWithUUID + "\"}"; + } + else { + //noinspection ResultOfMethodCallIgnored + tempFile.delete(); + } + } - return "{\"error\": \"Failed to upload the file.\"}"; + return "{\"error\": \"Failed to upload the file.\"}"; + }, + executor); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandler.java index 163e5832470f8..4dede3a512dd7 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.history.ArchivedJson; @@ -31,6 +32,8 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns the aggregated user accumulators of a job. @@ -39,8 +42,8 @@ public class JobAccumulatorsHandler extends AbstractExecutionGraphRequestHandler private static final String JOB_ACCUMULATORS_REST_PATH = "/jobs/:jobid/accumulators"; - public JobAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public JobAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -49,8 +52,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { - return createJobAccumulatorsJson(graph); + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createJobAccumulatorsJson(graph); + } catch (IOException e) { + throw new FlinkFutureException("Could not create job accumulators json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java index 513dc0896076a..1a7d86880493d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandler.java @@ -20,11 +20,14 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler for the CANCEL request. @@ -36,7 +39,8 @@ public class JobCancellationHandler extends AbstractJsonRequestHandler { private final Time timeout; - public JobCancellationHandler(Time timeout) { + public JobCancellationHandler(Executor executor, Time timeout) { + super(executor); this.timeout = Preconditions.checkNotNull(timeout); } @@ -46,19 +50,23 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - try { - JobID jobId = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid"))); - if (jobManagerGateway != null) { - jobManagerGateway.cancelJob(jobId, timeout); - return "{}"; - } - else { - throw new Exception("No connection to the leading JobManager."); - } - } - catch (Exception e) { - throw new Exception("Failed to cancel the job with id: " + pathParams.get("jobid") + e.getMessage(), e); - } + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.supplyAsync( + () -> { + try { + JobID jobId = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid"))); + if (jobManagerGateway != null) { + jobManagerGateway.cancelJob(jobId, timeout); + return "{}"; + } + else { + throw new Exception("No connection to the leading JobManager."); + } + } + catch (Exception e) { + throw new FlinkFutureException("Failed to cancel the job with id: " + pathParams.get("jobid"), e); + } + }, + executor); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java index 9b474aa788b85..4e4144745f300 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlers.java @@ -24,12 +24,13 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.runtime.checkpoint.CheckpointCoordinator; +import org.apache.flink.runtime.concurrent.FlinkFutureException; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.messages.JobManagerMessages.CancelJobWithSavepoint; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.NotFoundException; -import org.apache.flink.util.FlinkException; import org.apache.flink.shaded.netty4.io.netty.buffer.Unpooled; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHttpResponse; @@ -140,48 +141,48 @@ public String[] getPaths() { @Override @SuppressWarnings("unchecked") - public FullHttpResponse handleRequest( + public CompletableFuture handleRequest( Map pathParams, Map queryParams, - JobManagerGateway jobManagerGateway) throws Exception { + JobManagerGateway jobManagerGateway) { - try { - if (jobManagerGateway != null) { - JobID jobId = JobID.fromHexString(pathParams.get("jobid")); - final Optional optGraph; + if (jobManagerGateway != null) { + JobID jobId = JobID.fromHexString(pathParams.get("jobid")); + final CompletableFuture> graphFuture; - try { - optGraph = currentGraphs.getExecutionGraph(jobId, jobManagerGateway); - } catch (Exception e) { - throw new FlinkException("Could not retrieve the execution with jobId " + jobId + " from the JobManager.", e); - } + graphFuture = currentGraphs.getExecutionGraph(jobId, jobManagerGateway); - final AccessExecutionGraph graph = optGraph.orElseThrow( - () -> new NotFoundException("Could not find ExecutionGraph with jobId " + jobId + '.')); + return graphFuture.thenApplyAsync( + (Optional optGraph) -> { + final AccessExecutionGraph graph = optGraph.orElseThrow( + () -> new FlinkFutureException( + new NotFoundException("Could not find ExecutionGraph with jobId " + jobId + '.'))); - CheckpointCoordinator coord = graph.getCheckpointCoordinator(); - if (coord == null) { - throw new Exception("Cannot find CheckpointCoordinator for job."); - } + CheckpointCoordinator coord = graph.getCheckpointCoordinator(); + if (coord == null) { + throw new FlinkFutureException(new Exception("Cannot find CheckpointCoordinator for job.")); + } - String targetDirectory = pathParams.get("targetDirectory"); - if (targetDirectory == null) { - if (defaultSavepointDirectory == null) { - throw new IllegalStateException("No savepoint directory configured. " + + String targetDirectory = pathParams.get("targetDirectory"); + if (targetDirectory == null) { + if (defaultSavepointDirectory == null) { + throw new IllegalStateException("No savepoint directory configured. " + "You can either specify a directory when triggering this savepoint or " + "configure a cluster-wide default via key '" + CoreOptions.SAVEPOINT_DIRECTORY.key() + "'."); - } else { - targetDirectory = defaultSavepointDirectory; + } else { + targetDirectory = defaultSavepointDirectory; + } } - } - return handleNewRequest(jobManagerGateway, jobId, targetDirectory, coord.getCheckpointTimeout()); - } else { - throw new Exception("No connection to the leading JobManager."); - } - } catch (Exception e) { - throw new Exception("Failed to cancel the job: " + e.getMessage(), e); + try { + return handleNewRequest(jobManagerGateway, jobId, targetDirectory, coord.getCheckpointTimeout()); + } catch (IOException e) { + throw new FlinkFutureException("Could not cancel job with savepoint.", e); + } + }, executor); + } else { + return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager.")); } } @@ -288,64 +289,63 @@ public String[] getPaths() { @Override @SuppressWarnings("unchecked") - public FullHttpResponse handleRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - try { - if (jobManagerGateway != null) { - JobID jobId = JobID.fromHexString(pathParams.get("jobid")); - long requestId = Long.parseLong(pathParams.get("requestId")); + public CompletableFuture handleRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + JobID jobId = JobID.fromHexString(pathParams.get("jobid")); + long requestId = Long.parseLong(pathParams.get("requestId")); - synchronized (lock) { - Object result = completed.remove(requestId); - - if (result != null) { - // Add to recent history - recentlyCompleted.add(new Tuple2<>(requestId, result)); - if (recentlyCompleted.size() > NUM_GHOST_REQUEST_IDS) { - recentlyCompleted.remove(); - } + return CompletableFuture.supplyAsync( + () -> { + try { + synchronized (lock) { + Object result = completed.remove(requestId); + + if (result != null) { + // Add to recent history + recentlyCompleted.add(new Tuple2<>(requestId, result)); + if (recentlyCompleted.size() > NUM_GHOST_REQUEST_IDS) { + recentlyCompleted.remove(); + } - if (result.getClass() == String.class) { - String savepointPath = (String) result; - return createSuccessResponse(requestId, savepointPath); - } else { - Throwable cause = (Throwable) result; - return createFailureResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, requestId, cause.getMessage()); - } - } else { - // Check in-progress - Long inProgressRequestId = inProgress.get(jobId); - if (inProgressRequestId != null) { - // Sanity check - if (inProgressRequestId == requestId) { - return createInProgressResponse(requestId); + if (result.getClass() == String.class) { + String savepointPath = (String) result; + return createSuccessResponse(requestId, savepointPath); } else { - String msg = "Request ID does not belong to JobID"; - return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, msg); + Throwable cause = (Throwable) result; + return createFailureResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, requestId, cause.getMessage()); } - } - - // Check recent history - for (Tuple2 recent : recentlyCompleted) { - if (recent.f0 == requestId) { - if (recent.f1.getClass() == String.class) { - String savepointPath = (String) recent.f1; - return createSuccessResponse(requestId, savepointPath); + } else { + // Check in-progress + Long inProgressRequestId = inProgress.get(jobId); + if (inProgressRequestId != null) { + // Sanity check + if (inProgressRequestId == requestId) { + return createInProgressResponse(requestId); } else { - Throwable cause = (Throwable) recent.f1; - return createFailureResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, requestId, cause.getMessage()); + String msg = "Request ID does not belong to JobID"; + return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, msg); } } - } - return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, "Unknown job/request ID"); + // Check recent history + for (Tuple2 recent : recentlyCompleted) { + if (recent.f0 == requestId) { + if (recent.f1.getClass() == String.class) { + String savepointPath = (String) recent.f1; + return createSuccessResponse(requestId, savepointPath); + } else { + Throwable cause = (Throwable) recent.f1; + return createFailureResponse(HttpResponseStatus.INTERNAL_SERVER_ERROR, requestId, cause.getMessage()); + } + } + } + + return createFailureResponse(HttpResponseStatus.BAD_REQUEST, requestId, "Unknown job/request ID"); + } } + } catch (Exception e) { + throw new FlinkFutureException("Could not handle in progress request.", e); } - } else { - throw new Exception("No connection to the leading JobManager."); - } - } catch (Exception e) { - throw new Exception("Failed to cancel the job: " + e.getMessage(), e); - } + }); } private FullHttpResponse createSuccessResponse(long requestId, String savepointPath) throws IOException { diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java index 72cf8b7f7e304..0b15b375fdf81 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.ArchivedExecutionConfig; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.history.ArchivedJson; @@ -31,6 +32,8 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns the execution config of a job. @@ -39,8 +42,8 @@ public class JobConfigHandler extends AbstractExecutionGraphRequestHandler { private static final String JOB_CONFIG_REST_PATH = "/jobs/:jobid/config"; - public JobConfigHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public JobConfigHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -49,8 +52,17 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { - return createJobConfigJson(graph); + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createJobConfigJson(graph); + } catch (IOException e) { + throw new FlinkFutureException("Could not write job config json.", e); + } + }, + executor); + } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java index 87ac7c3c6777c..8a50f87080d6f 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -39,6 +40,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns details about a job. This includes: @@ -57,8 +60,8 @@ public class JobDetailsHandler extends AbstractExecutionGraphRequestHandler { private final MetricFetcher fetcher; - public JobDetailsHandler(ExecutionGraphHolder executionGraphHolder, MetricFetcher fetcher) { - super(executionGraphHolder); + public JobDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) { + super(executionGraphHolder, executor); this.fetcher = fetcher; } @@ -68,8 +71,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { - return createJobDetailsJson(graph, fetcher); + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createJobDetailsJson(graph, fetcher); + } catch (IOException e) { + throw new FlinkFutureException("Could not create job details json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java index e31299b3e53d8..6ffd4432bf909 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; @@ -35,6 +36,8 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns the configuration of a job. @@ -45,8 +48,8 @@ public class JobExceptionsHandler extends AbstractExecutionGraphRequestHandler { static final int MAX_NUMBER_EXCEPTION_TO_REPORT = 20; - public JobExceptionsHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public JobExceptionsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -55,8 +58,17 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { - return createJobExceptionsJson(graph); + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createJobExceptionsJson(graph); + } catch (IOException e) { + throw new FlinkFutureException("Could not create job exceptions json.", e); + } + }, + executor + ); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java index e2437e67d421b..cb6d8c0a557a8 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandler.java @@ -19,12 +19,16 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import com.fasterxml.jackson.core.JsonGenerator; +import java.io.IOException; import java.io.StringWriter; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Returns the Job Manager's configuration. @@ -35,7 +39,8 @@ public class JobManagerConfigHandler extends AbstractJsonRequestHandler { private final Configuration config; - public JobManagerConfigHandler(Configuration config) { + public JobManagerConfigHandler(Executor executor, Configuration config) { + super(executor); this.config = config; } @@ -45,31 +50,38 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.supplyAsync( + () -> { + try { + StringWriter writer = new StringWriter(); + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - gen.writeStartArray(); - for (String key : config.keySet()) { - gen.writeStartObject(); - gen.writeStringField("key", key); + gen.writeStartArray(); + for (String key : config.keySet()) { + gen.writeStartObject(); + gen.writeStringField("key", key); - // Mask key values which contain sensitive information - if (key.toLowerCase().contains("password")) { - String value = config.getString(key, null); - if (value != null) { - value = "******"; - } - gen.writeStringField("value", value); - } - else { - gen.writeStringField("value", config.getString(key, null)); - } - gen.writeEndObject(); - } - gen.writeEndArray(); + // Mask key values which contain sensitive information + if (key.toLowerCase().contains("password")) { + String value = config.getString(key, null); + if (value != null) { + value = "******"; + } + gen.writeStringField("value", value); + } else { + gen.writeStringField("value", config.getString(key, null)); + } + gen.writeEndObject(); + } + gen.writeEndArray(); - gen.close(); - return writer.toString(); + gen.close(); + return writer.toString(); + } catch (IOException e) { + throw new FlinkFutureException("Could not write configuration.", e); + } + }, + executor); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandler.java index d17b6bb596081..b3a9dd5925271 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandler.java @@ -27,6 +27,8 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns the JSON program plan of a job graph. @@ -35,8 +37,8 @@ public class JobPlanHandler extends AbstractExecutionGraphRequestHandler { private static final String JOB_PLAN_REST_PATH = "/jobs/:jobid/plan"; - public JobPlanHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public JobPlanHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -45,8 +47,8 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { - return graph.getJsonPlan(); + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { + return CompletableFuture.completedFuture(graph.getJsonPlan()); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java index 352673412f141..f63403f3baf1c 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandler.java @@ -20,11 +20,14 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.util.Preconditions; import org.apache.flink.util.StringUtils; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler for the STOP request. @@ -36,7 +39,8 @@ public class JobStoppingHandler extends AbstractJsonRequestHandler { private final Time timeout; - public JobStoppingHandler(Time timeout) { + public JobStoppingHandler(Executor executor, Time timeout) { + super(executor); this.timeout = Preconditions.checkNotNull(timeout); } @@ -46,19 +50,23 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - try { - JobID jobId = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid"))); - if (jobManagerGateway != null) { - jobManagerGateway.stopJob(jobId, timeout); - return "{}"; - } - else { - throw new Exception("No connection to the leading JobManager."); - } - } - catch (Exception e) { - throw new Exception("Failed to stop the job with id: " + pathParams.get("jobid") + e.getMessage(), e); - } + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.supplyAsync( + () -> { + try { + JobID jobId = new JobID(StringUtils.hexStringToByte(pathParams.get("jobid"))); + if (jobManagerGateway != null) { + jobManagerGateway.stopJob(jobId, timeout); + return "{}"; + } + else { + throw new Exception("No connection to the leading JobManager."); + } + } + catch (Exception e) { + throw new FlinkFutureException("Failed to stop the job with id: " + pathParams.get("jobid") + '.', e); + } + }, + executor); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandler.java index 8e90dfcf4b9a8..9c613fff781a4 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; @@ -33,6 +34,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns the accummulators for a given vertex. @@ -41,8 +44,8 @@ public class JobVertexAccumulatorsHandler extends AbstractJobVertexRequestHandle private static final String JOB_VERTEX_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/accumulators"; - public JobVertexAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public JobVertexAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -51,8 +54,17 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionJobVertex jobVertex, Map params) throws Exception { - return createVertexAccumulatorsJson(jobVertex); + public CompletableFuture handleRequest(AccessExecutionJobVertex jobVertex, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createVertexAccumulatorsJson(jobVertex); + } catch (IOException e) { + throw new FlinkFutureException("Could not create job vertex accumulators json.", e); + } + }, + executor); + } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandler.java index cde8ca9ef988f..963153fbe11f4 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ArchivedExecutionJobVertex; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; @@ -27,8 +28,11 @@ import com.fasterxml.jackson.core.JsonGenerator; +import java.io.IOException; import java.io.StringWriter; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import scala.Option; @@ -51,10 +55,11 @@ public class JobVertexBackPressureHandler extends AbstractJobVertexRequestHandle public JobVertexBackPressureHandler( ExecutionGraphHolder executionGraphHolder, + Executor executor, BackPressureStatsTracker backPressureStatsTracker, int refreshInterval) { - super(executionGraphHolder); + super(executionGraphHolder, executor); this.backPressureStatsTracker = checkNotNull(backPressureStatsTracker, "Stats tracker"); checkArgument(refreshInterval >= 0, "Negative timeout"); this.refreshInterval = refreshInterval; @@ -66,11 +71,11 @@ public String[] getPaths() { } @Override - public String handleRequest( + public CompletableFuture handleRequest( AccessExecutionJobVertex accessJobVertex, - Map params) throws Exception { + Map params) { if (accessJobVertex instanceof ArchivedExecutionJobVertex) { - return ""; + return CompletableFuture.completedFuture(""); } ExecutionJobVertex jobVertex = (ExecutionJobVertex) accessJobVertex; try (StringWriter writer = new StringWriter(); @@ -116,7 +121,9 @@ public String handleRequest( gen.writeEndObject(); gen.close(); - return writer.toString(); + return CompletableFuture.completedFuture(writer.toString()); + } catch (IOException e) { + return FutureUtils.completedExceptionally(e); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java index 7757fdde92a21..bd1745c662b5a 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -39,6 +40,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * A request handler that provides the details of a job vertex, including id, name, parallelism, @@ -50,8 +53,8 @@ public class JobVertexDetailsHandler extends AbstractJobVertexRequestHandler { private final MetricFetcher fetcher; - public JobVertexDetailsHandler(ExecutionGraphHolder executionGraphHolder, MetricFetcher fetcher) { - super(executionGraphHolder); + public JobVertexDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) { + super(executionGraphHolder, executor); this.fetcher = fetcher; } @@ -61,8 +64,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionJobVertex jobVertex, Map params) throws Exception { - return createVertexDetailsJson(jobVertex, params.get("jobid"), fetcher); + public CompletableFuture handleRequest(AccessExecutionJobVertex jobVertex, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createVertexDetailsJson(jobVertex, params.get("jobid"), fetcher); + } catch (IOException e) { + throw new FlinkFutureException("Could not write the vertex details json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java index a612782884b86..0827720b2340a 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -41,6 +42,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * A request handler that provides the details of a job vertex, including id, name, and the @@ -52,8 +55,8 @@ public class JobVertexTaskManagersHandler extends AbstractJobVertexRequestHandle private final MetricFetcher fetcher; - public JobVertexTaskManagersHandler(ExecutionGraphHolder executionGraphHolder, MetricFetcher fetcher) { - super(executionGraphHolder); + public JobVertexTaskManagersHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) { + super(executionGraphHolder, executor); this.fetcher = fetcher; } @@ -63,8 +66,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionJobVertex jobVertex, Map params) throws Exception { - return createVertexDetailsByTaskManagerJson(jobVertex, params.get("jobid"), fetcher); + public CompletableFuture handleRequest(AccessExecutionJobVertex jobVertex, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createVertexDetailsByTaskManagerJson(jobVertex, params.get("jobid"), fetcher); + } catch (IOException e) { + throw new FlinkFutureException("Could not create TaskManager json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java index 079be8f37fce9..8ca785f29b998 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/RequestHandler.java @@ -19,11 +19,11 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.runtime.jobmaster.JobManagerGateway; -import org.apache.flink.runtime.webmonitor.NotFoundException; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse; import java.util.Map; +import java.util.concurrent.CompletableFuture; /** * Base interface for all request handlers. @@ -44,13 +44,8 @@ public interface RequestHandler { * @param jobManagerGateway to talk to the JobManager. * * @return The full http response. - * - * @throws Exception Handlers may forward exceptions. Exceptions of type - * {@link NotFoundException} will cause a HTTP 404 - * response with the exception message, other exceptions will cause a HTTP 500 response - * with the exception stack trace. */ - FullHttpResponse handleRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception; + CompletableFuture handleRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway); /** * Returns an array of REST URL's under which this handler can be registered. diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandler.java index 28e9ddf6dc169..301b21701283f 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandler.java @@ -23,6 +23,8 @@ import org.apache.flink.runtime.webmonitor.metrics.MetricFetcher; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler providing details about a single task execution attempt. @@ -31,8 +33,8 @@ public class SubtaskCurrentAttemptDetailsHandler extends SubtaskExecutionAttempt public static final String SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum"; - public SubtaskCurrentAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, MetricFetcher fetcher) { - super(executionGraphHolder, fetcher); + public SubtaskCurrentAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) { + super(executionGraphHolder, executor, fetcher); } @Override @@ -41,7 +43,7 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionVertex vertex, Map params) throws Exception { + public CompletableFuture handleRequest(AccessExecutionVertex vertex, Map params) { return handleRequest(vertex.getCurrentExecutionAttempt(), params); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandler.java index 171277ffb7022..3c0d1d938a2e1 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.executiongraph.AccessExecution; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -35,6 +36,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Base class for request handlers whose response depends on a specific job vertex (defined @@ -44,8 +47,8 @@ public class SubtaskExecutionAttemptAccumulatorsHandler extends AbstractSubtaskA private static final String SUBTASK_ATTEMPT_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt/accumulators"; - public SubtaskExecutionAttemptAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public SubtaskExecutionAttemptAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -54,8 +57,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecution execAttempt, Map params) throws Exception { - return createAttemptAccumulatorsJson(execAttempt); + public CompletableFuture handleRequest(AccessExecution execAttempt, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createAttemptAccumulatorsJson(execAttempt); + } catch (IOException e) { + throw new FlinkFutureException("Could not create accumulator json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java index 37c0e50a2071e..ad836dfa73158 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecution; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; @@ -40,6 +41,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import static org.apache.flink.runtime.webmonitor.handlers.SubtaskCurrentAttemptDetailsHandler.SUBTASK_CURRENT_ATTEMPT_DETAILS_REST_PATH; @@ -52,8 +55,8 @@ public class SubtaskExecutionAttemptDetailsHandler extends AbstractSubtaskAttemp private final MetricFetcher fetcher; - public SubtaskExecutionAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, MetricFetcher fetcher) { - super(executionGraphHolder); + public SubtaskExecutionAttemptDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, MetricFetcher fetcher) { + super(executionGraphHolder, executor); this.fetcher = fetcher; } @@ -63,8 +66,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecution execAttempt, Map params) throws Exception { - return createAttemptDetailsJson(execAttempt, params.get("jobid"), params.get("vertexid"), fetcher); + public CompletableFuture handleRequest(AccessExecution execAttempt, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createAttemptDetailsJson(execAttempt, params.get("jobid"), params.get("vertexid"), fetcher); + } catch (IOException e) { + throw new FlinkFutureException("Could not create attempt details json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java index 64bdfb45b3f26..814254883c76d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; @@ -35,6 +36,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns the accumulators for all subtasks of job vertex. @@ -43,8 +46,8 @@ public class SubtasksAllAccumulatorsHandler extends AbstractJobVertexRequestHand private static final String SUBTASKS_ALL_ACCUMULATORS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasks/accumulators"; - public SubtasksAllAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public SubtasksAllAccumulatorsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -53,8 +56,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionJobVertex jobVertex, Map params) throws Exception { - return createSubtasksAccumulatorsJson(jobVertex); + public CompletableFuture handleRequest(AccessExecutionJobVertex jobVertex, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createSubtasksAccumulatorsJson(jobVertex); + } catch (IOException e) { + throw new FlinkFutureException("Could not create subtasks accumulator json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java index ea885873f879f..d76620614bd3b 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -35,6 +36,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns the state transition timestamps for all subtasks, plus their @@ -44,8 +47,8 @@ public class SubtasksTimesHandler extends AbstractJobVertexRequestHandler { private static final String SUBTASK_TIMES_REST_PATH = "/jobs/:jobid/vertices/:vertexid/subtasktimes"; - public SubtasksTimesHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public SubtasksTimesHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -54,8 +57,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionJobVertex jobVertex, Map params) throws Exception { - return createSubtaskTimesJson(jobVertex); + public CompletableFuture handleRequest(AccessExecutionJobVertex jobVertex, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createSubtaskTimesJson(jobVertex); + } catch (IOException e) { + throw new FlinkFutureException("Could not write subtask time json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java index a8ab7a3d68418..9f83ed07daa10 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandler.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.FlinkFutureException; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobManagerGateway; @@ -28,14 +30,14 @@ import com.fasterxml.jackson.core.JsonGenerator; +import java.io.IOException; import java.io.StringWriter; -import java.util.ArrayList; import java.util.Collection; -import java.util.List; +import java.util.Collections; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.Executor; import static java.util.Objects.requireNonNull; @@ -53,7 +55,8 @@ public class TaskManagersHandler extends AbstractJsonRequestHandler { private final MetricFetcher fetcher; - public TaskManagersHandler(Time timeout, MetricFetcher fetcher) { + public TaskManagersHandler(Executor executor, Time timeout, MetricFetcher fetcher) { + super(executor); this.timeout = requireNonNull(timeout); this.fetcher = fetcher; } @@ -64,134 +67,139 @@ public String[] getPaths() { } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - try { - if (jobManagerGateway != null) { - // whether one task manager's metrics are requested, or all task manager, we - // return them in an array. This avoids unnecessary code complexity. - // If only one task manager is requested, we only fetch one task manager metrics. - final List instances = new ArrayList<>(); - if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) { - try { - InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(pathParams.get(TASK_MANAGER_ID_KEY))); - CompletableFuture> tmInstanceFuture = jobManagerGateway.requestTaskManagerInstance(instanceID, timeout); - - Optional instance = tmInstanceFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); - - instance.ifPresent(instances::add); - } - // this means the id string was invalid. Keep the list empty. - catch (IllegalArgumentException e){ - // do nothing. - } - } else { - CompletableFuture> tmInstancesFuture = jobManagerGateway.requestTaskManagerInstances(timeout); - - Collection tmInstances = tmInstancesFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); - instances.addAll(tmInstances); - } + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + if (jobManagerGateway != null) { + // whether one task manager's metrics are requested, or all task manager, we + // return them in an array. This avoids unnecessary code complexity. + // If only one task manager is requested, we only fetch one task manager metrics. + if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) { + InstanceID instanceID = new InstanceID(StringUtils.hexStringToByte(pathParams.get(TASK_MANAGER_ID_KEY))); + CompletableFuture> tmInstanceFuture = jobManagerGateway.requestTaskManagerInstance(instanceID, timeout); + + return tmInstanceFuture.thenApplyAsync( + (Optional optTaskManager) -> { + try { + return writeTaskManagersJson( + optTaskManager.map(Collections::singleton).orElse(Collections.emptySet()), + pathParams); + } catch (IOException e) { + throw new FlinkFutureException("Could not write TaskManagers JSON.", e); + } + }, + executor); + } else { + CompletableFuture> tmInstancesFuture = jobManagerGateway.requestTaskManagerInstances(timeout); + + return tmInstancesFuture.thenApplyAsync( + (Collection taskManagers) -> { + try { + return writeTaskManagersJson(taskManagers, pathParams); + } catch (IOException e) { + throw new FlinkFutureException("Could not write TaskManagers JSON.", e); + } + }, + executor); + } + } + else { + return FutureUtils.completedExceptionally(new Exception("No connection to the leading JobManager.")); + } + } - StringWriter writer = new StringWriter(); - JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); - - gen.writeStartObject(); - gen.writeArrayFieldStart("taskmanagers"); - - for (Instance instance : instances) { - gen.writeStartObject(); - gen.writeStringField("id", instance.getId().toString()); - gen.writeStringField("path", instance.getTaskManagerGateway().getAddress()); - gen.writeNumberField("dataPort", instance.getTaskManagerLocation().dataPort()); - gen.writeNumberField("timeSinceLastHeartbeat", instance.getLastHeartBeat()); - gen.writeNumberField("slotsNumber", instance.getTotalNumberOfSlots()); - gen.writeNumberField("freeSlots", instance.getNumberOfAvailableSlots()); - gen.writeNumberField("cpuCores", instance.getResources().getNumberOfCPUCores()); - gen.writeNumberField("physicalMemory", instance.getResources().getSizeOfPhysicalMemory()); - gen.writeNumberField("freeMemory", instance.getResources().getSizeOfJvmHeap()); - gen.writeNumberField("managedMemory", instance.getResources().getSizeOfManagedMemory()); - - // only send metrics when only one task manager requests them. - if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) { - fetcher.update(); - MetricStore.TaskManagerMetricStore metrics = fetcher.getMetricStore().getTaskManagerMetricStore(instance.getId().toString()); - if (metrics != null) { - gen.writeObjectFieldStart("metrics"); - long heapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Used", "0")); - long heapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Committed", "0")); - long heapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Max", "0")); - - gen.writeNumberField("heapCommitted", heapCommitted); - gen.writeNumberField("heapUsed", heapUsed); - gen.writeNumberField("heapMax", heapTotal); - - long nonHeapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Used", "0")); - long nonHeapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Committed", "0")); - long nonHeapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Max", "0")); - - gen.writeNumberField("nonHeapCommitted", nonHeapCommitted); - gen.writeNumberField("nonHeapUsed", nonHeapUsed); - gen.writeNumberField("nonHeapMax", nonHeapTotal); - - gen.writeNumberField("totalCommitted", heapCommitted + nonHeapCommitted); - gen.writeNumberField("totalUsed", heapUsed + nonHeapUsed); - gen.writeNumberField("totalMax", heapTotal + nonHeapTotal); - - long directCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.Count", "0")); - long directUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.MemoryUsed", "0")); - long directMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.TotalCapacity", "0")); - - gen.writeNumberField("directCount", directCount); - gen.writeNumberField("directUsed", directUsed); - gen.writeNumberField("directMax", directMax); - - long mappedCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.Count", "0")); - long mappedUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.MemoryUsed", "0")); - long mappedMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.TotalCapacity", "0")); - - gen.writeNumberField("mappedCount", mappedCount); - gen.writeNumberField("mappedUsed", mappedUsed); - gen.writeNumberField("mappedMax", mappedMax); - - long memorySegmentsAvailable = Long.valueOf(metrics.getMetric("Status.Network.AvailableMemorySegments", "0")); - long memorySegmentsTotal = Long.valueOf(metrics.getMetric("Status.Network.TotalMemorySegments", "0")); - - gen.writeNumberField("memorySegmentsAvailable", memorySegmentsAvailable); - gen.writeNumberField("memorySegmentsTotal", memorySegmentsTotal); - - gen.writeArrayFieldStart("garbageCollectors"); - - for (String gcName : metrics.garbageCollectorNames) { - String count = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Count", null); - String time = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Time", null); - if (count != null && time != null) { - gen.writeStartObject(); - gen.writeStringField("name", gcName); - gen.writeNumberField("count", Long.valueOf(count)); - gen.writeNumberField("time", Long.valueOf(time)); - gen.writeEndObject(); - } - } - - gen.writeEndArray(); + private String writeTaskManagersJson(Collection instances, Map pathParams) throws IOException { + StringWriter writer = new StringWriter(); + JsonGenerator gen = JsonFactory.JACKSON_FACTORY.createGenerator(writer); + + gen.writeStartObject(); + gen.writeArrayFieldStart("taskmanagers"); + + for (Instance instance : instances) { + gen.writeStartObject(); + gen.writeStringField("id", instance.getId().toString()); + gen.writeStringField("path", instance.getTaskManagerGateway().getAddress()); + gen.writeNumberField("dataPort", instance.getTaskManagerLocation().dataPort()); + gen.writeNumberField("timeSinceLastHeartbeat", instance.getLastHeartBeat()); + gen.writeNumberField("slotsNumber", instance.getTotalNumberOfSlots()); + gen.writeNumberField("freeSlots", instance.getNumberOfAvailableSlots()); + gen.writeNumberField("cpuCores", instance.getResources().getNumberOfCPUCores()); + gen.writeNumberField("physicalMemory", instance.getResources().getSizeOfPhysicalMemory()); + gen.writeNumberField("freeMemory", instance.getResources().getSizeOfJvmHeap()); + gen.writeNumberField("managedMemory", instance.getResources().getSizeOfManagedMemory()); + + // only send metrics when only one task manager requests them. + if (pathParams.containsKey(TASK_MANAGER_ID_KEY)) { + fetcher.update(); + MetricStore.TaskManagerMetricStore metrics = fetcher.getMetricStore().getTaskManagerMetricStore(instance.getId().toString()); + if (metrics != null) { + gen.writeObjectFieldStart("metrics"); + long heapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Used", "0")); + long heapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Committed", "0")); + long heapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Heap.Max", "0")); + + gen.writeNumberField("heapCommitted", heapCommitted); + gen.writeNumberField("heapUsed", heapUsed); + gen.writeNumberField("heapMax", heapTotal); + + long nonHeapUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Used", "0")); + long nonHeapCommitted = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Committed", "0")); + long nonHeapTotal = Long.valueOf(metrics.getMetric("Status.JVM.Memory.NonHeap.Max", "0")); + + gen.writeNumberField("nonHeapCommitted", nonHeapCommitted); + gen.writeNumberField("nonHeapUsed", nonHeapUsed); + gen.writeNumberField("nonHeapMax", nonHeapTotal); + + gen.writeNumberField("totalCommitted", heapCommitted + nonHeapCommitted); + gen.writeNumberField("totalUsed", heapUsed + nonHeapUsed); + gen.writeNumberField("totalMax", heapTotal + nonHeapTotal); + + long directCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.Count", "0")); + long directUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.MemoryUsed", "0")); + long directMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Direct.TotalCapacity", "0")); + + gen.writeNumberField("directCount", directCount); + gen.writeNumberField("directUsed", directUsed); + gen.writeNumberField("directMax", directMax); + + long mappedCount = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.Count", "0")); + long mappedUsed = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.MemoryUsed", "0")); + long mappedMax = Long.valueOf(metrics.getMetric("Status.JVM.Memory.Mapped.TotalCapacity", "0")); + + gen.writeNumberField("mappedCount", mappedCount); + gen.writeNumberField("mappedUsed", mappedUsed); + gen.writeNumberField("mappedMax", mappedMax); + + long memorySegmentsAvailable = Long.valueOf(metrics.getMetric("Status.Network.AvailableMemorySegments", "0")); + long memorySegmentsTotal = Long.valueOf(metrics.getMetric("Status.Network.TotalMemorySegments", "0")); + + gen.writeNumberField("memorySegmentsAvailable", memorySegmentsAvailable); + gen.writeNumberField("memorySegmentsTotal", memorySegmentsTotal); + + gen.writeArrayFieldStart("garbageCollectors"); + + for (String gcName : metrics.garbageCollectorNames) { + String count = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Count", null); + String time = metrics.getMetric("Status.JVM.GarbageCollector." + gcName + ".Time", null); + if (count != null && time != null) { + gen.writeStartObject(); + gen.writeStringField("name", gcName); + gen.writeNumberField("count", Long.valueOf(count)); + gen.writeNumberField("time", Long.valueOf(time)); gen.writeEndObject(); } } + gen.writeEndArray(); gen.writeEndObject(); } - - gen.writeEndArray(); - gen.writeEndObject(); - - gen.close(); - return writer.toString(); - } - else { - throw new Exception("No connection to the leading JobManager."); } + + gen.writeEndObject(); } - catch (Exception e) { - throw new RuntimeException("Failed to fetch list of all task managers: " + e.getMessage(), e); - } + + gen.writeEndArray(); + gen.writeEndObject(); + + gen.close(); + return writer.toString(); } } diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandler.java index d4c9b2a65b432..3affd7c9f74be 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers.checkpoints; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; @@ -34,6 +35,8 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Handler that returns a job's snapshotting settings. @@ -42,8 +45,8 @@ public class CheckpointConfigHandler extends AbstractExecutionGraphRequestHandle private static final String CHECKPOINT_CONFIG_REST_PATH = "/jobs/:jobid/checkpoints/config"; - public CheckpointConfigHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public CheckpointConfigHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -52,8 +55,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { - return createCheckpointConfigJson(graph); + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createCheckpointConfigJson(graph); + } catch (IOException e) { + throw new FlinkFutureException("Could not create checkpoint config json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandler.java index 664744b05198b..96cc3e0637515 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandler.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats; import org.apache.flink.runtime.checkpoint.FailedCheckpointStats; import org.apache.flink.runtime.checkpoint.TaskStateStats; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.handlers.AbstractExecutionGraphRequestHandler; @@ -40,6 +41,8 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Request handler that returns checkpoint stats for a single job vertex. @@ -50,8 +53,8 @@ public class CheckpointStatsDetailsHandler extends AbstractExecutionGraphRequest private final CheckpointStatsCache cache; - public CheckpointStatsDetailsHandler(ExecutionGraphHolder executionGraphHolder, CheckpointStatsCache cache) { - super(executionGraphHolder); + public CheckpointStatsDetailsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, CheckpointStatsCache cache) { + super(executionGraphHolder, executor); this.cache = cache; } @@ -61,30 +64,38 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { - long checkpointId = parseCheckpointId(params); - if (checkpointId == -1) { - return "{}"; - } - - CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot(); - if (snapshot == null) { - return "{}"; - } - - AbstractCheckpointStats checkpoint = snapshot.getHistory().getCheckpointById(checkpointId); - - if (checkpoint != null) { - cache.tryAdd(checkpoint); - } else { - checkpoint = cache.tryGet(checkpointId); - - if (checkpoint == null) { - return "{}"; - } - } - - return createCheckpointDetailsJson(checkpoint); + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { + return CompletableFuture.supplyAsync( + () -> { + long checkpointId = parseCheckpointId(params); + if (checkpointId == -1) { + return "{}"; + } + + CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot(); + if (snapshot == null) { + return "{}"; + } + + AbstractCheckpointStats checkpoint = snapshot.getHistory().getCheckpointById(checkpointId); + + if (checkpoint != null) { + cache.tryAdd(checkpoint); + } else { + checkpoint = cache.tryGet(checkpointId); + + if (checkpoint == null) { + return "{}"; + } + } + + try { + return createCheckpointDetailsJson(checkpoint); + } catch (IOException e) { + throw new FlinkFutureException("Could not create checkpoint details json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java index d116c56362b48..045248b41591c 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsSubtasksHandler.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.checkpoint.MinMaxAvgStats; import org.apache.flink.runtime.checkpoint.SubtaskStateStats; import org.apache.flink.runtime.checkpoint.TaskStateStats; +import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmaster.JobManagerGateway; @@ -43,6 +44,8 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import static org.apache.flink.runtime.webmonitor.handlers.checkpoints.CheckpointStatsHandler.writeMinMaxAvg; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -57,8 +60,8 @@ public class CheckpointStatsDetailsSubtasksHandler extends AbstractExecutionGrap private final CheckpointStatsCache cache; - public CheckpointStatsDetailsSubtasksHandler(ExecutionGraphHolder executionGraphHolder, CheckpointStatsCache cache) { - super(executionGraphHolder); + public CheckpointStatsDetailsSubtasksHandler(ExecutionGraphHolder executionGraphHolder, Executor executor, CheckpointStatsCache cache) { + super(executionGraphHolder, executor); this.cache = checkNotNull(cache); } @@ -68,28 +71,28 @@ public String[] getPaths() { } @Override - public String handleJsonRequest( - Map pathParams, - Map queryParams, - JobManagerGateway jobManagerGateway) throws Exception { + public CompletableFuture handleJsonRequest( + Map pathParams, + Map queryParams, + JobManagerGateway jobManagerGateway) { return super.handleJsonRequest(pathParams, queryParams, jobManagerGateway); } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { long checkpointId = CheckpointStatsDetailsHandler.parseCheckpointId(params); if (checkpointId == -1) { - return "{}"; + return CompletableFuture.completedFuture("{}"); } JobVertexID vertexId = AbstractJobVertexRequestHandler.parseJobVertexId(params); if (vertexId == null) { - return "{}"; + return CompletableFuture.completedFuture("{}"); } CheckpointStatsSnapshot snapshot = graph.getCheckpointStatsSnapshot(); if (snapshot == null) { - return "{}"; + return CompletableFuture.completedFuture("{}"); } AbstractCheckpointStats checkpoint = snapshot.getHistory().getCheckpointById(checkpointId); @@ -100,16 +103,20 @@ public String handleRequest(AccessExecutionGraph graph, Map para checkpoint = cache.tryGet(checkpointId); if (checkpoint == null) { - return "{}"; + return CompletableFuture.completedFuture("{}"); } } TaskStateStats taskStats = checkpoint.getTaskStateStats(vertexId); if (taskStats == null) { - return "{}"; + return CompletableFuture.completedFuture("{}"); } - return createSubtaskCheckpointDetailsJson(checkpoint, taskStats); + try { + return CompletableFuture.completedFuture(createSubtaskCheckpointDetailsJson(checkpoint, taskStats)); + } catch (IOException e) { + return FutureUtils.completedExceptionally(e); + } } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandler.java index a86c5fdef1ff1..a60aee08ed23d 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandler.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.checkpoint.FailedCheckpointStats; import org.apache.flink.runtime.checkpoint.MinMaxAvgStats; import org.apache.flink.runtime.checkpoint.RestoredCheckpointStats; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.handlers.AbstractExecutionGraphRequestHandler; @@ -43,6 +44,8 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Handler that returns checkpoint statistics for a job. @@ -51,8 +54,8 @@ public class CheckpointStatsHandler extends AbstractExecutionGraphRequestHandler private static final String CHECKPOINT_STATS_REST_PATH = "/jobs/:jobid/checkpoints"; - public CheckpointStatsHandler(ExecutionGraphHolder executionGraphHolder) { - super(executionGraphHolder); + public CheckpointStatsHandler(ExecutionGraphHolder executionGraphHolder, Executor executor) { + super(executionGraphHolder, executor); } @Override @@ -61,8 +64,16 @@ public String[] getPaths() { } @Override - public String handleRequest(AccessExecutionGraph graph, Map params) throws Exception { - return createCheckpointStatsJson(graph); + public CompletableFuture handleRequest(AccessExecutionGraph graph, Map params) { + return CompletableFuture.supplyAsync( + () -> { + try { + return createCheckpointStatsJson(graph); + } catch (IOException e) { + throw new FlinkFutureException("Could not create checkpoint stats json.", e); + } + }, + executor); } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandler.java index b95f2c456d765..cf286ce1b1cc8 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.metrics; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.jobmaster.JobManagerGateway; import org.apache.flink.runtime.webmonitor.handlers.AbstractJsonRequestHandler; import org.apache.flink.runtime.webmonitor.handlers.JsonFactory; @@ -28,6 +29,8 @@ import java.io.IOException; import java.io.StringWriter; import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; /** * Abstract request handler that returns a list of all available metrics or the values for a set of metrics. @@ -43,17 +46,27 @@ public abstract class AbstractMetricsHandler extends AbstractJsonRequestHandler { private final MetricFetcher fetcher; - public AbstractMetricsHandler(MetricFetcher fetcher) { + public AbstractMetricsHandler(Executor executor, MetricFetcher fetcher) { + super(executor); this.fetcher = Preconditions.checkNotNull(fetcher); } @Override - public String handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) throws Exception { - fetcher.update(); - String requestedMetricsList = queryParams.get("get"); - return requestedMetricsList != null - ? getMetricsValues(pathParams, requestedMetricsList) - : getAvailableMetricsList(pathParams); + public CompletableFuture handleJsonRequest(Map pathParams, Map queryParams, JobManagerGateway jobManagerGateway) { + return CompletableFuture.supplyAsync( + () -> { + fetcher.update(); + String requestedMetricsList = queryParams.get("get"); + try { + return requestedMetricsList != null + ? getMetricsValues(pathParams, requestedMetricsList) + : getAvailableMetricsList(pathParams); + } catch (IOException e) { + throw new FlinkFutureException("Could not retrieve metrics.", e); + } + }, + executor); + } /** diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandler.java index 7252d8a5d9c35..2bd668391cd62 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.metrics; import java.util.Map; +import java.util.concurrent.Executor; /** * Request handler that returns for the job manager a list of all available metrics or the values for a set of metrics. @@ -35,8 +36,8 @@ public class JobManagerMetricsHandler extends AbstractMetricsHandler { private static final String JOBMANAGER_METRICS_REST_PATH = "/jobmanager/metrics"; - public JobManagerMetricsHandler(MetricFetcher fetcher) { - super(fetcher); + public JobManagerMetricsHandler(Executor executor, MetricFetcher fetcher) { + super(executor, fetcher); } @Override diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandler.java index a193457c7b220..e5e25002dd056 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.metrics; import java.util.Map; +import java.util.concurrent.Executor; /** * Request handler that returns for a given job a list of all available metrics or the values for a set of metrics. @@ -35,8 +36,8 @@ public class JobMetricsHandler extends AbstractMetricsHandler { public static final String PARAMETER_JOB_ID = "jobid"; private static final String JOB_METRICS_REST_PATH = "/jobs/:jobid/metrics"; - public JobMetricsHandler(MetricFetcher fetcher) { - super(fetcher); + public JobMetricsHandler(Executor executor, MetricFetcher fetcher) { + super(executor, fetcher); } @Override diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandler.java index e893da4b29013..1d2cd841752b0 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandler.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.metrics; import java.util.Map; +import java.util.concurrent.Executor; /** * Request handler that returns for a given task a list of all available metrics or the values for a set of metrics. @@ -35,8 +36,8 @@ public class JobVertexMetricsHandler extends AbstractMetricsHandler { public static final String PARAMETER_VERTEX_ID = "vertexid"; private static final String JOB_VERTEX_METRICS_REST_PATH = "/jobs/:jobid/vertices/:vertexid/metrics"; - public JobVertexMetricsHandler(MetricFetcher fetcher) { - super(fetcher); + public JobVertexMetricsHandler(Executor executor, MetricFetcher fetcher) { + super(executor, fetcher); } @Override diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java index bcc62cb7d796a..fde16fcc149c3 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandler.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.webmonitor.handlers.TaskManagersHandler; import java.util.Map; +import java.util.concurrent.Executor; /** * Request handler that returns for a given task manager a list of all available metrics or the values for a set of metrics. @@ -37,8 +38,8 @@ public class TaskManagerMetricsHandler extends AbstractMetricsHandler { private static final String TASKMANAGER_METRICS_REST_PATH = "/taskmanagers/:taskmanagerid/metrics"; - public TaskManagerMetricsHandler(MetricFetcher fetcher) { - super(fetcher); + public TaskManagerMetricsHandler(Executor executor, MetricFetcher fetcher) { + super(executor, fetcher); } @Override diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandlerTest.java index 865385f648567..69ee762d9097d 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/ClusterOverviewHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.Executors; import org.junit.Assert; import org.junit.Test; @@ -29,7 +30,7 @@ public class ClusterOverviewHandlerTest { @Test public void testGetPaths() { - ClusterOverviewHandler handler = new ClusterOverviewHandler(Time.seconds(0L)); + ClusterOverviewHandler handler = new ClusterOverviewHandler(Executors.directExecutor(), Time.seconds(0L)); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/overview", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandlerTest.java index ea26f5d9ad7bf..6061e4bcaf597 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobIdsHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.Executors; import org.junit.Assert; import org.junit.Test; @@ -29,7 +30,7 @@ public class CurrentJobIdsHandlerTest { @Test public void testGetPaths() { - CurrentJobIdsHandler handler = new CurrentJobIdsHandler(Time.seconds(0L)); + CurrentJobIdsHandler handler = new CurrentJobIdsHandler(Executors.directExecutor(), Time.seconds(0L)); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandlerTest.java index 64360d369154e..ccfafd445e692 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/CurrentJobsOverviewHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.messages.webmonitor.JobDetails; @@ -66,17 +67,17 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - CurrentJobsOverviewHandler handlerAll = new CurrentJobsOverviewHandler(Time.seconds(0L), true, true); + CurrentJobsOverviewHandler handlerAll = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), true, true); String[] pathsAll = handlerAll.getPaths(); Assert.assertEquals(1, pathsAll.length); Assert.assertEquals("/joboverview", pathsAll[0]); - CurrentJobsOverviewHandler handlerRunning = new CurrentJobsOverviewHandler(Time.seconds(0L), true, false); + CurrentJobsOverviewHandler handlerRunning = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), true, false); String[] pathsRunning = handlerRunning.getPaths(); Assert.assertEquals(1, pathsRunning.length); Assert.assertEquals("/joboverview/running", pathsRunning[0]); - CurrentJobsOverviewHandler handlerCompleted = new CurrentJobsOverviewHandler(Time.seconds(0L), false, true); + CurrentJobsOverviewHandler handlerCompleted = new CurrentJobsOverviewHandler(Executors.directExecutor(), Time.seconds(0L), false, true); String[] pathsCompleted = handlerCompleted.getPaths(); Assert.assertEquals(1, pathsCompleted.length); Assert.assertEquals("/joboverview/completed", pathsCompleted[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandlerTest.java index d17b55fb7d5ae..22b3e5e1d326f 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/DashboardConfigHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.util.EnvironmentInformation; import org.apache.flink.runtime.webmonitor.utils.ArchivedJobGenerationUtils; @@ -33,7 +34,7 @@ public class DashboardConfigHandlerTest { @Test public void testGetPaths() { - DashboardConfigHandler handler = new DashboardConfigHandler(10000L); + DashboardConfigHandler handler = new DashboardConfigHandler(Executors.directExecutor(), 10000L); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/config", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandlerTest.java index a498cf2f8f547..97091cff8bd74 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarAccessDeniedHandlerTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; + import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import org.junit.Assert; @@ -31,7 +33,7 @@ public class JarAccessDeniedHandlerTest { @Test public void testGetPaths() { - JarAccessDeniedHandler handler = new JarAccessDeniedHandler(); + JarAccessDeniedHandler handler = new JarAccessDeniedHandler(Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(5, paths.length); List pathsList = Lists.newArrayList(paths); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java index bcbb1eaba946c..a067d8feda515 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarDeleteHandlerTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; + import org.junit.Assert; import org.junit.Test; @@ -27,7 +29,7 @@ public class JarDeleteHandlerTest { @Test public void testGetPaths() { - JarDeleteHandler handler = new JarDeleteHandler(null); + JarDeleteHandler handler = new JarDeleteHandler(Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jars/:jarid", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandlerTest.java index 863c248840961..5da4913818991 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarListHandlerTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; + import org.junit.Assert; import org.junit.Test; @@ -27,7 +29,7 @@ public class JarListHandlerTest { @Test public void testGetPaths() { - JarListHandler handler = new JarListHandler(null); + JarListHandler handler = new JarListHandler(Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jars", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerTest.java index a3ded83028a0c..f5ed339a05609 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; + import org.junit.Assert; import org.junit.Test; @@ -27,7 +29,7 @@ public class JarPlanHandlerTest { @Test public void testGetPaths() { - JarPlanHandler handler = new JarPlanHandler(null); + JarPlanHandler handler = new JarPlanHandler(Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jars/:jarid/plan", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java index 82aa87a3f58b0..67dad135c0c16 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.concurrent.Executors; import org.junit.Assert; import org.junit.Test; @@ -30,7 +31,7 @@ public class JarRunHandlerTest { @Test public void testGetPaths() { - JarRunHandler handler = new JarRunHandler(null, Time.seconds(0L), new Configuration()); + JarRunHandler handler = new JarRunHandler(Executors.directExecutor(), null, Time.seconds(0L), new Configuration()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jars/:jarid/run", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java index e57ca342aeeda..ea8b5248d6079 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarUploadHandlerTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; + import org.junit.Assert; import org.junit.Test; @@ -27,7 +29,7 @@ public class JarUploadHandlerTest { @Test public void testGetPaths() { - JarUploadHandler handler = new JarUploadHandler(null); + JarUploadHandler handler = new JarUploadHandler(Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jars/upload", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandlerTest.java index fe55f51af3a78..5510fedb90155 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobAccumulatorsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.history.ArchivedJson; @@ -54,7 +55,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - JobAccumulatorsHandler handler = new JobAccumulatorsHandler(mock(ExecutionGraphHolder.class)); + JobAccumulatorsHandler handler = new JobAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/accumulators", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandlerTest.java index a436b2d2f1dca..86c5295df4645 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -33,7 +34,7 @@ public class JobCancellationHandlerTest { @Test public void testGetPaths() { - JobCancellationHandler handler = new JobCancellationHandler(TestingUtils.TIMEOUT()); + JobCancellationHandler handler = new JobCancellationHandler(Executors.directExecutor(), TestingUtils.TIMEOUT()); String[] paths = handler.getPaths(); Assert.assertEquals(2, paths.length); List pathsList = Lists.newArrayList(paths); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java index b48ee665ffd96..529d13052e181 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobCancellationWithSavepointHandlersTest.java @@ -92,7 +92,7 @@ public void testAskTimeoutEqualsCheckpointTimeout() throws Exception { ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class); ExecutionGraph graph = mock(ExecutionGraph.class); CheckpointCoordinator coord = mock(CheckpointCoordinator.class); - when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(Optional.of(graph)); + when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph))); when(graph.getCheckpointCoordinator()).thenReturn(coord); when(coord.getCheckpointTimeout()).thenReturn(timeout); @@ -121,7 +121,7 @@ public void testSavepointDirectoryConfiguration() throws Exception { ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class); ExecutionGraph graph = mock(ExecutionGraph.class); CheckpointCoordinator coord = mock(CheckpointCoordinator.class); - when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(Optional.of(graph)); + when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph))); when(graph.getCheckpointCoordinator()).thenReturn(coord); when(coord.getCheckpointTimeout()).thenReturn(timeout); @@ -152,7 +152,7 @@ public void testSavepointDirectoryConfiguration() throws Exception { handler = handlers.getTriggerHandler(); try { - handler.handleRequest(params, Collections.emptyMap(), jobManager); + handler.handleRequest(params, Collections.emptyMap(), jobManager).get(); fail("Did not throw expected test Exception"); } catch (Exception e) { IllegalStateException cause = (IllegalStateException) e.getCause(); @@ -169,7 +169,7 @@ public void testTriggerNewRequest() throws Exception { ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class); ExecutionGraph graph = mock(ExecutionGraph.class); CheckpointCoordinator coord = mock(CheckpointCoordinator.class); - when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(Optional.of(graph)); + when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph))); when(graph.getCheckpointCoordinator()).thenReturn(coord); JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor); @@ -187,7 +187,7 @@ public void testTriggerNewRequest() throws Exception { when(jobManager.cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class))).thenReturn(successfulCancelWithSavepoint); // Trigger - FullHttpResponse response = trigger.handleRequest(params, Collections.emptyMap(), jobManager); + FullHttpResponse response = trigger.handleRequest(params, Collections.emptyMap(), jobManager).get(); verify(jobManager).cancelJobWithSavepoint(eq(jobId), eq("custom-directory"), any(Time.class)); @@ -206,7 +206,7 @@ public void testTriggerNewRequest() throws Exception { assertEquals(location, root.get("location").asText()); // Trigger again - response = trigger.handleRequest(params, Collections.emptyMap(), jobManager); + response = trigger.handleRequest(params, Collections.emptyMap(), jobManager).get(); assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus()); assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE)); assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH)); @@ -225,7 +225,7 @@ public void testTriggerNewRequest() throws Exception { // Query progress params.put("requestId", "1"); - response = progress.handleRequest(params, Collections.emptyMap(), jobManager); + response = progress.handleRequest(params, Collections.emptyMap(), jobManager).get(); assertEquals(HttpResponseStatus.ACCEPTED, response.getStatus()); assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE)); assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH)); @@ -239,7 +239,7 @@ public void testTriggerNewRequest() throws Exception { // Complete successfulCancelWithSavepoint.complete("_path-savepoint_"); - response = progress.handleRequest(params, Collections.emptyMap(), jobManager); + response = progress.handleRequest(params, Collections.emptyMap(), jobManager).get(); assertEquals(HttpResponseStatus.CREATED, response.getStatus()); assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE)); @@ -255,7 +255,7 @@ public void testTriggerNewRequest() throws Exception { // Query again, keep recent history - response = progress.handleRequest(params, Collections.emptyMap(), jobManager); + response = progress.handleRequest(params, Collections.emptyMap(), jobManager).get(); assertEquals(HttpResponseStatus.CREATED, response.getStatus()); assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE)); @@ -272,7 +272,7 @@ public void testTriggerNewRequest() throws Exception { // Query for unknown request params.put("requestId", "9929"); - response = progress.handleRequest(params, Collections.emptyMap(), jobManager); + response = progress.handleRequest(params, Collections.emptyMap(), jobManager).get(); assertEquals(HttpResponseStatus.BAD_REQUEST, response.getStatus()); assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE)); assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH)); @@ -295,7 +295,7 @@ public void testFailedCancellation() throws Exception { ExecutionGraphHolder holder = mock(ExecutionGraphHolder.class); ExecutionGraph graph = mock(ExecutionGraph.class); CheckpointCoordinator coord = mock(CheckpointCoordinator.class); - when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(Optional.of(graph)); + when(holder.getExecutionGraph(eq(jobId), any(JobManagerGateway.class))).thenReturn(CompletableFuture.completedFuture(Optional.of(graph))); when(graph.getCheckpointCoordinator()).thenReturn(coord); JobCancellationWithSavepointHandlers handlers = new JobCancellationWithSavepointHandlers(holder, executor); @@ -319,7 +319,7 @@ public void testFailedCancellation() throws Exception { // Query progress params.put("requestId", "1"); - FullHttpResponse response = progress.handleRequest(params, Collections.emptyMap(), jobManager); + FullHttpResponse response = progress.handleRequest(params, Collections.emptyMap(), jobManager).get(); assertEquals(HttpResponseStatus.INTERNAL_SERVER_ERROR, response.getStatus()); assertEquals("application/json; charset=UTF-8", response.headers().get(HttpHeaders.Names.CONTENT_TYPE)); assertEquals(Integer.toString(response.content().readableBytes()), response.headers().get(HttpHeaders.Names.CONTENT_LENGTH)); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandlerTest.java index 104b0a3011234..1c08ae860090f 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobConfigHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.ArchivedExecutionConfig; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.history.ArchivedJson; @@ -55,7 +56,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - JobConfigHandler handler = new JobConfigHandler(mock(ExecutionGraphHolder.class)); + JobConfigHandler handler = new JobConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/config", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandlerTest.java index bfeb40aa9df86..ee0498e486f03 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobDetailsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -68,7 +69,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - JobDetailsHandler handler = new JobDetailsHandler(mock(ExecutionGraphHolder.class), null); + JobDetailsHandler handler = new JobDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(2, paths.length); List pathsList = Lists.newArrayList(paths); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java index f54ab062a7f60..6e0f9187018f7 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobExceptionsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; @@ -58,7 +59,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - JobExceptionsHandler handler = new JobExceptionsHandler(mock(ExecutionGraphHolder.class)); + JobExceptionsHandler handler = new JobExceptionsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/exceptions", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandlerTest.java index 8e16e8afe6600..94fd5a85ae7fc 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobManagerConfigHandlerTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; + import org.junit.Assert; import org.junit.Test; @@ -27,7 +29,7 @@ public class JobManagerConfigHandlerTest { @Test public void testGetPaths() { - JobManagerConfigHandler handler = new JobManagerConfigHandler(null); + JobManagerConfigHandler handler = new JobManagerConfigHandler(Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobmanager/config", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandlerTest.java index 17b4c4484aa10..4a934ecd1cd98 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobPlanHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.history.ArchivedJson; @@ -51,7 +52,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - JobPlanHandler handler = new JobPlanHandler(mock(ExecutionGraphHolder.class)); + JobPlanHandler handler = new JobPlanHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/plan", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandlerTest.java index 89bf4260089ce..8c05c83c7b678 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobStoppingHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.util.TestLogger; @@ -34,7 +35,7 @@ public class JobStoppingHandlerTest extends TestLogger { @Test public void testGetPaths() { - JobStoppingHandler handler = new JobStoppingHandler(TestingUtils.TIMEOUT()); + JobStoppingHandler handler = new JobStoppingHandler(Executors.directExecutor(), TestingUtils.TIMEOUT()); String[] paths = handler.getPaths(); Assert.assertEquals(2, paths.length); List pathsList = Lists.newArrayList(paths); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandlerTest.java index b7af323279b07..5af9aa635664f 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexAccumulatorsHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; @@ -57,7 +58,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - JobVertexAccumulatorsHandler handler = new JobVertexAccumulatorsHandler(mock(ExecutionGraphHolder.class)); + JobVertexAccumulatorsHandler handler = new JobVertexAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/accumulators", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandlerTest.java index d2ac0d62271d7..0d15e084f52ad 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexBackPressureHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.webmonitor.BackPressureStatsTracker; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; @@ -46,7 +47,7 @@ public class JobVertexBackPressureHandlerTest { @Test public void testGetPaths() { - JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(mock(ExecutionGraphHolder.class), mock(BackPressureStatsTracker.class), 0); + JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), mock(BackPressureStatsTracker.class), 0); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/backpressure", paths[0]); @@ -63,10 +64,11 @@ public void testResponseNoStatsAvailable() throws Exception { JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler( mock(ExecutionGraphHolder.class), + Executors.directExecutor(), statsTracker, 9999); - String response = handler.handleRequest(jobVertex, Collections.emptyMap()); + String response = handler.handleRequest(jobVertex, Collections.emptyMap()).get(); ObjectMapper mapper = new ObjectMapper(); JsonNode rootNode = mapper.readTree(response); @@ -96,10 +98,11 @@ public void testResponseStatsAvailable() throws Exception { JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler( mock(ExecutionGraphHolder.class), + Executors.directExecutor(), statsTracker, 9999); - String response = handler.handleRequest(jobVertex, Collections.emptyMap()); + String response = handler.handleRequest(jobVertex, Collections.emptyMap()).get(); ObjectMapper mapper = new ObjectMapper(); JsonNode rootNode = mapper.readTree(response); @@ -157,10 +160,11 @@ public void testResponsePassedRefreshInterval() throws Exception { JobVertexBackPressureHandler handler = new JobVertexBackPressureHandler( mock(ExecutionGraphHolder.class), + Executors.directExecutor(), statsTracker, 0); // <----- refresh interval should fire immediately - String response = handler.handleRequest(jobVertex, Collections.emptyMap()); + String response = handler.handleRequest(jobVertex, Collections.emptyMap()).get(); ObjectMapper mapper = new ObjectMapper(); JsonNode rootNode = mapper.readTree(response); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandlerTest.java index bc4fe9ce5055e..1b8d9aa9d1102 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexDetailsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -59,7 +60,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - JobVertexDetailsHandler handler = new JobVertexDetailsHandler(mock(ExecutionGraphHolder.class), null); + JobVertexDetailsHandler handler = new JobVertexDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandlerTest.java index d5d877aa9dd1d..badb9528aeb60 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JobVertexTaskManagersHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -61,7 +62,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - JobVertexTaskManagersHandler handler = new JobVertexTaskManagersHandler(mock(ExecutionGraphHolder.class), null); + JobVertexTaskManagersHandler handler = new JobVertexTaskManagersHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/taskmanagers", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandlerTest.java index d992b85771763..a80bac93010a7 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskCurrentAttemptDetailsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.junit.Assert; @@ -31,7 +32,7 @@ public class SubtaskCurrentAttemptDetailsHandlerTest { @Test public void testGetPaths() { - SubtaskCurrentAttemptDetailsHandler handler = new SubtaskCurrentAttemptDetailsHandler(mock(ExecutionGraphHolder.class), null); + SubtaskCurrentAttemptDetailsHandler handler = new SubtaskCurrentAttemptDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandlerTest.java index ce8e72fad4ca5..6773fd4875313 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptAccumulatorsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecution; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; @@ -64,7 +65,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - SubtaskExecutionAttemptAccumulatorsHandler handler = new SubtaskExecutionAttemptAccumulatorsHandler(mock(ExecutionGraphHolder.class)); + SubtaskExecutionAttemptAccumulatorsHandler handler = new SubtaskExecutionAttemptAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt/accumulators", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandlerTest.java index e1fbf9271662a..7777d2dd21585 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtaskExecutionAttemptDetailsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecution; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; @@ -73,7 +74,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - SubtaskExecutionAttemptDetailsHandler handler = new SubtaskExecutionAttemptDetailsHandler(mock(ExecutionGraphHolder.class), null); + SubtaskExecutionAttemptDetailsHandler handler = new SubtaskExecutionAttemptDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), null); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/:subtasknum/attempts/:attempt", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandlerTest.java index f33da80642fc0..7b400da37d936 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksAllAccumulatorsHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; @@ -58,7 +59,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - SubtasksAllAccumulatorsHandler handler = new SubtasksAllAccumulatorsHandler(mock(ExecutionGraphHolder.class)); + SubtasksAllAccumulatorsHandler handler = new SubtasksAllAccumulatorsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasks/accumulators", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandlerTest.java index 548efaf5646c7..31c2212204b6f 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/SubtasksTimesHandlerTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.webmonitor.handlers; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecution; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; @@ -59,7 +60,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - SubtasksTimesHandler handler = new SubtasksTimesHandler(mock(ExecutionGraphHolder.class)); + SubtasksTimesHandler handler = new SubtasksTimesHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/subtasktimes", paths[0]); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandlerTest.java index afc27644561cd..e3a71a1a57e75 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/TaskManagersHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers; import org.apache.flink.api.common.time.Time; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -33,7 +34,7 @@ public class TaskManagersHandlerTest { @Test public void testGetPaths() { - TaskManagersHandler handler = new TaskManagersHandler(Time.seconds(0L), null); + TaskManagersHandler handler = new TaskManagersHandler(Executors.directExecutor(), Time.seconds(0L), null); String[] paths = handler.getPaths(); Assert.assertEquals(2, paths.length); List pathsList = Lists.newArrayList(paths); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java index ce943b107c9f2..47298be28cf1e 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointConfigHandlerTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.webmonitor.handlers.checkpoints; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.ExternalizedCheckpointSettings; @@ -79,7 +80,7 @@ public void testArchiver() throws IOException { @Test public void testGetPaths() { - CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class)); + CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/checkpoints/config", paths[0]); @@ -95,8 +96,8 @@ public void testSimpleConfig() throws Exception { AccessExecutionGraph graph = graphAndSettings.graph; JobCheckpointingSettings settings = graphAndSettings.snapshottingSettings; - CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class)); - String json = handler.handleRequest(graph, Collections.emptyMap()); + CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); + String json = handler.handleRequest(graph, Collections.emptyMap()).get(); ObjectMapper mapper = new ObjectMapper(); JsonNode rootNode = mapper.readTree(json); @@ -121,8 +122,8 @@ public void testAtLeastOnce() throws Exception { AccessExecutionGraph graph = graphAndSettings.graph; - CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class)); - String json = handler.handleRequest(graph, Collections.emptyMap()); + CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); + String json = handler.handleRequest(graph, Collections.emptyMap()).get(); ObjectMapper mapper = new ObjectMapper(); JsonNode rootNode = mapper.readTree(json); @@ -140,8 +141,8 @@ public void testEnabledExternalizedCheckpointSettings() throws Exception { AccessExecutionGraph graph = graphAndSettings.graph; ExternalizedCheckpointSettings externalizedSettings = graphAndSettings.externalizedSettings; - CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class)); - String json = handler.handleRequest(graph, Collections.emptyMap()); + CheckpointConfigHandler handler = new CheckpointConfigHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); + String json = handler.handleRequest(graph, Collections.emptyMap()).get(); ObjectMapper mapper = new ObjectMapper(); JsonNode externalizedNode = mapper.readTree(json).get("externalization"); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandlerTest.java index 0259aa57ddf1e..f16d6231e3ff5 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsDetailsHandlerTest.java @@ -28,6 +28,7 @@ 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.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; @@ -101,7 +102,7 @@ public void testArchiver() throws IOException { @Test public void testGetPaths() { - CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/checkpoints/details/:checkpointid", paths[0]); @@ -113,10 +114,10 @@ public void testGetPaths() { @Test public void testIllegalCheckpointId() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); - CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "illegal checkpoint"); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); assertEquals("{}", json); } @@ -127,8 +128,8 @@ public void testIllegalCheckpointId() throws Exception { @Test public void testNoCheckpointIdParam() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); - CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); - String json = handler.handleRequest(graph, Collections.emptyMap()); + CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); + String json = handler.handleRequest(graph, Collections.emptyMap()).get(); assertEquals("{}", json); } @@ -147,10 +148,10 @@ public void testCheckpointNotFound() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot); - CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "123"); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); assertEquals("{}", json); verify(history, times(1)).getCheckpointById(anyLong()); @@ -318,10 +319,10 @@ private static JsonNode triggerRequest(AbstractCheckpointStats checkpoint) throw AccessExecutionGraph graph = mock(AccessExecutionGraph.class); when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot); - CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsHandler handler = new CheckpointStatsDetailsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "123"); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); ObjectMapper mapper = new ObjectMapper(); return mapper.readTree(json); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandlerTest.java index 9425a4ca1b3de..ed73a620fa56a 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsHandlerTest.java @@ -31,6 +31,7 @@ import org.apache.flink.runtime.checkpoint.MinMaxAvgStats; import org.apache.flink.runtime.checkpoint.PendingCheckpointStats; import org.apache.flink.runtime.checkpoint.RestoredCheckpointStats; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; import org.apache.flink.runtime.webmonitor.history.ArchivedJson; @@ -86,7 +87,7 @@ public void testArchiver() throws IOException { @Test public void testGetPaths() { - CheckpointStatsHandler handler = new CheckpointStatsHandler(mock(ExecutionGraphHolder.class)); + CheckpointStatsHandler handler = new CheckpointStatsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/checkpoints", paths[0]); @@ -99,8 +100,8 @@ public void testGetPaths() { public void testCheckpointStatsRequest() throws Exception { TestCheckpointStats testCheckpointStats = createTestCheckpointStats(); - CheckpointStatsHandler handler = new CheckpointStatsHandler(mock(ExecutionGraphHolder.class)); - String json = handler.handleRequest(testCheckpointStats.graph, Collections.emptyMap()); + CheckpointStatsHandler handler = new CheckpointStatsHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor()); + String json = handler.handleRequest(testCheckpointStats.graph, Collections.emptyMap()).get(); ObjectMapper mapper = new ObjectMapper(); JsonNode rootNode = mapper.readTree(json); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java index b8eb715d2e1d4..9c5e168c6f9cd 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/checkpoints/CheckpointStatsSubtaskDetailsHandlerTest.java @@ -27,6 +27,7 @@ import org.apache.flink.runtime.checkpoint.PendingCheckpointStats; import org.apache.flink.runtime.checkpoint.SubtaskStateStats; import org.apache.flink.runtime.checkpoint.TaskStateStats; +import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.webmonitor.ExecutionGraphHolder; @@ -100,7 +101,7 @@ public void testArchiver() throws Exception { @Test public void testGetPaths() { - CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/checkpoints/details/:checkpointid/subtasks/:vertexid", paths[0]); @@ -149,10 +150,10 @@ public void testSubtaskRequestNoSummary() throws Exception { @Test public void testIllegalCheckpointId() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); - CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "illegal checkpoint"); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); assertEquals("{}", json); } @@ -163,8 +164,8 @@ public void testIllegalCheckpointId() throws Exception { @Test public void testNoCheckpointIdParam() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); - CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); - String json = handler.handleRequest(graph, Collections.emptyMap()); + CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); + String json = handler.handleRequest(graph, Collections.emptyMap()).get(); assertEquals("{}", json); } @@ -183,11 +184,11 @@ public void testCheckpointNotFound() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot); - CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "123"); params.put("vertexid", new JobVertexID().toString()); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); assertEquals("{}", json); verify(history, times(1)).getCheckpointById(anyLong()); @@ -199,11 +200,11 @@ public void testCheckpointNotFound() throws Exception { @Test public void testIllegalJobVertexIdParam() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); - CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "1"); params.put("vertexid", "illegal vertex id"); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); assertEquals("{}", json); } @@ -214,10 +215,10 @@ public void testIllegalJobVertexIdParam() throws Exception { @Test public void testNoJobVertexIdParam() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); - CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "1"); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); assertEquals("{}", json); } @@ -238,11 +239,11 @@ public void testJobVertexNotFound() throws Exception { AccessExecutionGraph graph = mock(AccessExecutionGraph.class); when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot); - CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "123"); params.put("vertexid", new JobVertexID().toString()); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); assertEquals("{}", json); verify(inProgress, times(1)).getTaskStateStats(any(JobVertexID.class)); @@ -259,11 +260,11 @@ private static JsonNode triggerRequest(AbstractCheckpointStats checkpoint) throw AccessExecutionGraph graph = mock(AccessExecutionGraph.class); when(graph.getCheckpointStatsSnapshot()).thenReturn(snapshot); - CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), new CheckpointStatsCache(0)); + CheckpointStatsDetailsSubtasksHandler handler = new CheckpointStatsDetailsSubtasksHandler(mock(ExecutionGraphHolder.class), Executors.directExecutor(), new CheckpointStatsCache(0)); Map params = new HashMap<>(); params.put("checkpointid", "123"); params.put("vertexid", new JobVertexID().toString()); - String json = handler.handleRequest(graph, params); + String json = handler.handleRequest(graph, params).get(); ObjectMapper mapper = new ObjectMapper(); return mapper.readTree(json); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandlerTest.java index 90e032d971e51..5296d335e7099 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/AbstractMetricsHandlerTest.java @@ -49,7 +49,7 @@ public void testHandleRequest() throws Exception { TestingUtils.TIMEOUT()); MetricStoreTest.setupStore(fetcher.getMetricStore()); - JobVertexMetricsHandler handler = new JobVertexMetricsHandler(fetcher); + JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); Map queryParams = new HashMap<>(); @@ -58,7 +58,7 @@ public void testHandleRequest() throws Exception { pathParams.put("vertexid", "taskid"); // get list of available metrics - String availableList = handler.handleJsonRequest(pathParams, queryParams, null); + String availableList = handler.handleJsonRequest(pathParams, queryParams, null).get(); assertEquals("[" + "{\"id\":\"8.opname.abc.metric5\"}," + @@ -69,7 +69,7 @@ public void testHandleRequest() throws Exception { // get value for a single metric queryParams.put("get", "8.opname.abc.metric5"); - String metricValue = handler.handleJsonRequest(pathParams, queryParams, null); + String metricValue = handler.handleJsonRequest(pathParams, queryParams, null).get(); assertEquals("[" + "{\"id\":\"8.opname.abc.metric5\",\"value\":\"4\"}" + @@ -80,7 +80,7 @@ public void testHandleRequest() throws Exception { // get values for multiple metrics queryParams.put("get", "8.opname.abc.metric5,8.abc.metric4"); - String metricValues = handler.handleJsonRequest(pathParams, queryParams, null); + String metricValues = handler.handleJsonRequest(pathParams, queryParams, null).get(); assertEquals("[" + "{\"id\":\"8.opname.abc.metric5\",\"value\":\"4\"}," + @@ -102,7 +102,7 @@ public void testInvalidListDoesNotFail() { TestingUtils.TIMEOUT()); MetricStoreTest.setupStore(fetcher.getMetricStore()); - JobVertexMetricsHandler handler = new JobVertexMetricsHandler(fetcher); + JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); Map queryParams = new HashMap<>(); @@ -114,7 +114,7 @@ public void testInvalidListDoesNotFail() { pathParams.put("jobid", "nonexistent"); try { - assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null)); + assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get()); } catch (Exception e) { fail(); } @@ -132,7 +132,7 @@ public void testInvalidGetDoesNotFail() { TestingUtils.TIMEOUT()); MetricStoreTest.setupStore(fetcher.getMetricStore()); - JobVertexMetricsHandler handler = new JobVertexMetricsHandler(fetcher); + JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); Map queryParams = new HashMap<>(); @@ -144,7 +144,7 @@ public void testInvalidGetDoesNotFail() { queryParams.put("get", ""); try { - assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null)); + assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get()); } catch (Exception e) { fail(e.getMessage()); } @@ -154,7 +154,7 @@ public void testInvalidGetDoesNotFail() { queryParams.put("get", "subindex.opname.abc.metric5"); try { - assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null)); + assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get()); } catch (Exception e) { fail(e.getMessage()); } @@ -164,7 +164,7 @@ public void testInvalidGetDoesNotFail() { queryParams.put("get", "subindex.opname.abc.nonexistant"); try { - assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null)); + assertEquals("", handler.handleJsonRequest(pathParams, queryParams, null).get()); } catch (Exception e) { fail(e.getMessage()); } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandlerTest.java index 994fc5eababbb..b02949a6a6174 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobManagerMetricsHandlerTest.java @@ -40,7 +40,7 @@ public class JobManagerMetricsHandlerTest extends TestLogger { @Test public void testGetPaths() { - JobManagerMetricsHandler handler = new JobManagerMetricsHandler(mock(MetricFetcher.class)); + JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class)); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobmanager/metrics", paths[0]); @@ -55,7 +55,7 @@ public void getMapFor() { TestingUtils.TIMEOUT()); MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore()); - JobManagerMetricsHandler handler = new JobManagerMetricsHandler(fetcher); + JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); @@ -73,7 +73,7 @@ public void getMapForNull() { TestingUtils.TIMEOUT()); MetricStore store = fetcher.getMetricStore(); - JobManagerMetricsHandler handler = new JobManagerMetricsHandler(fetcher); + JobManagerMetricsHandler handler = new JobManagerMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandlerTest.java index a35af222253be..569f772422718 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobMetricsHandlerTest.java @@ -41,7 +41,7 @@ public class JobMetricsHandlerTest extends TestLogger { @Test public void testGetPaths() { - JobMetricsHandler handler = new JobMetricsHandler(mock(MetricFetcher.class)); + JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class)); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/metrics", paths[0]); @@ -56,7 +56,7 @@ public void getMapFor() throws Exception { TestingUtils.TIMEOUT()); MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore()); - JobMetricsHandler handler = new JobMetricsHandler(fetcher); + JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); pathParams.put(PARAMETER_JOB_ID, "jobid"); @@ -75,7 +75,7 @@ public void getMapForNull() { TestingUtils.TIMEOUT()); MetricStore store = fetcher.getMetricStore(); - JobMetricsHandler handler = new JobMetricsHandler(fetcher); + JobMetricsHandler handler = new JobMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandlerTest.java index e84b11da3a2ee..e6bbd2e4c19a1 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/JobVertexMetricsHandlerTest.java @@ -42,7 +42,7 @@ public class JobVertexMetricsHandlerTest extends TestLogger { @Test public void testGetPaths() { - JobVertexMetricsHandler handler = new JobVertexMetricsHandler(mock(MetricFetcher.class)); + JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class)); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/jobs/:jobid/vertices/:vertexid/metrics", paths[0]); @@ -57,7 +57,7 @@ public void getMapFor() throws Exception { TestingUtils.TIMEOUT()); MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore()); - JobVertexMetricsHandler handler = new JobVertexMetricsHandler(fetcher); + JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); pathParams.put(PARAMETER_JOB_ID, "jobid"); @@ -79,7 +79,7 @@ public void getMapForNull() { TestingUtils.TIMEOUT()); MetricStore store = fetcher.getMetricStore(); - JobVertexMetricsHandler handler = new JobVertexMetricsHandler(fetcher); + JobVertexMetricsHandler handler = new JobVertexMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java index c20ea98cbd264..c4c1c7ab453c4 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/metrics/TaskManagerMetricsHandlerTest.java @@ -41,7 +41,7 @@ public class TaskManagerMetricsHandlerTest extends TestLogger { @Test public void testGetPaths() { - TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(mock(MetricFetcher.class)); + TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), mock(MetricFetcher.class)); String[] paths = handler.getPaths(); Assert.assertEquals(1, paths.length); Assert.assertEquals("/taskmanagers/:taskmanagerid/metrics", paths[0]); @@ -56,7 +56,7 @@ public void getMapFor() throws Exception { TestingUtils.TIMEOUT()); MetricStore store = MetricStoreTest.setupStore(fetcher.getMetricStore()); - TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(fetcher); + TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); pathParams.put(TASK_MANAGER_ID_KEY, "tmid"); @@ -75,7 +75,7 @@ public void getMapForNull() { TestingUtils.TIMEOUT()); MetricStore store = fetcher.getMetricStore(); - TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(fetcher); + TaskManagerMetricsHandler handler = new TaskManagerMetricsHandler(Executors.directExecutor(), fetcher); Map pathParams = new HashMap<>(); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala index 8c551a730c9f6..f0073db4dca53 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala @@ -2232,7 +2232,7 @@ object JobManager { new AkkaJobManagerRetriever(jobManagerSystem, timeout), new AkkaQueryServiceRetriever(jobManagerSystem, timeout), timeout, - jobManagerSystem.dispatcher) + futureExecutor) Option(webServer) } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index e62213008bcbf..41e56294c6d3e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -1094,7 +1094,7 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { fail("The slot request should have failed."); } catch (Exception e) { - assertTrue(ExceptionUtils.containsThrowable(e, SlotAllocationException.class)); + assertTrue(ExceptionUtils.findThrowable(e, SlotAllocationException.class).isPresent()); } // re-register From a3df5a2ca52880b6681446d95f1d916a01f55681 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 14 Aug 2017 14:48:33 +0200 Subject: [PATCH 084/129] [FLINK-7444] [rpc] Make external calls non-blocking Make all external calls from the RpcEndpoint's main thread non blocking by executing them as a runnable in an Executor. Make FatalErrorHandler calls non asynchronous This closes #4540. --- .../MesosResourceManager.java | 4 +- .../flink/runtime/dispatcher/Dispatcher.java | 22 ++---- .../flink/runtime/jobmaster/JobMaster.java | 25 +++---- .../runtime/minicluster/MiniCluster.java | 27 +++---- .../resourcemanager/ResourceManager.java | 73 ++++++++----------- .../flink/runtime/rpc/FatalErrorHandler.java | 11 +++ .../runtime/taskexecutor/TaskExecutor.java | 45 ++++-------- .../flink/yarn/YarnClusterDescriptorV2.java | 7 -- .../flink/yarn/YarnResourceManager.java | 2 +- 9 files changed, 89 insertions(+), 127 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 9a2ad42a48cb5..8a8f20842f959 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 @@ -416,7 +416,7 @@ public void startNewWorker(ResourceProfile resourceProfile) { // tell the launch coordinator to launch the new tasks launchCoordinator.tell(new LaunchCoordinator.Launch(Collections.singletonList((LaunchableTask) launchable)), selfActor); } catch (Exception ex) { - onFatalErrorAsync(new ResourceManagerException("Unable to request new workers.", ex)); + onFatalError(new ResourceManagerException("Unable to request new workers.", ex)); } } @@ -447,7 +447,7 @@ else if (workersBeingReturned.containsKey(resourceID)) { } } catch (Exception e) { - onFatalErrorAsync(new ResourceManagerException("Unable to release a worker.", e)); + onFatalError(new ResourceManagerException("Unable to release a worker.", e)); } } 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 29262cd973821..e7e1ec2124cf1 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 @@ -390,48 +390,40 @@ private DispatcherOnCompleteActions(JobID jobId) { public void jobFinished(JobExecutionResult result) { log.info("Job {} finished.", jobId); - runAsync(new Runnable() { - @Override - public void run() { + runAsync(() -> { try { removeJob(jobId, true); } catch (Exception e) { log.warn("Could not properly remove job {} from the dispatcher.", jobId, e); } - } - }); + }); } @Override public void jobFailed(Throwable cause) { log.info("Job {} failed.", jobId); - runAsync(new Runnable() { - @Override - public void run() { + runAsync(() -> { try { removeJob(jobId, true); } catch (Exception e) { log.warn("Could not properly remove job {} from the dispatcher.", jobId, e); } - } - }); + }); } @Override public void jobFinishedByOther() { log.info("Job {} was finished by other JobManager.", jobId); - runAsync(new Runnable() { - @Override - public void run() { + runAsync( + () -> { try { removeJob(jobId, false); } catch (Exception e) { log.warn("Could not properly remove job {} from the dispatcher.", jobId, e); } - } - }); + }); } } } 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 7e48da15f9e3e..3e66a3419bf60 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 @@ -877,20 +877,13 @@ private void suspendExecution(final Throwable cause) { //---------------------------------------------------------------------------------------------- private void handleFatalError(final Throwable cause) { - runAsync(new Runnable() { - @Override - public void run() { - log.error("Fatal error occurred on JobManager, cause: {}", cause.getMessage(), cause); - try { - shutDown(); - } catch (Exception e) { - cause.addSuppressed(e); - } + try { + log.error("Fatal error occurred on JobManager.", cause); + } catch (Throwable ignore) {} - errorHandler.onFatalError(cause); - } - }); + // The fatal error handler implementation should make sure that this call is non-blocking + errorHandler.onFatalError(cause); } private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, final Throwable error) { @@ -910,7 +903,7 @@ private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, fina Map accumulatorResults = executionGraph.getAccumulators(); JobExecutionResult result = new JobExecutionResult(jobID, 0L, accumulatorResults); - jobCompletionActions.jobFinished(result); + executor.execute(() -> jobCompletionActions.jobFinished(result)); } catch (Exception e) { log.error("Cannot fetch final accumulators for job {} ({})", jobName, jobID, e); @@ -920,7 +913,7 @@ private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, fina "The job is registered as 'FINISHED (successful), but this notification describes " + "a failure, since the resulting accumulators could not be fetched.", e); - jobCompletionActions.jobFailed(exception); + executor.execute(() ->jobCompletionActions.jobFailed(exception)); } break; @@ -928,7 +921,7 @@ private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, fina final JobExecutionException exception = new JobExecutionException( jobID, "Job was cancelled.", new Exception("The job was cancelled")); - jobCompletionActions.jobFailed(exception); + executor.execute(() -> jobCompletionActions.jobFailed(exception)); break; } @@ -936,7 +929,7 @@ private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, fina final Throwable unpackedError = SerializedThrowable.get(error, userCodeLoader); final JobExecutionException exception = new JobExecutionException( jobID, "Job execution failed.", unpackedError); - jobCompletionActions.jobFailed(exception); + executor.execute(() -> jobCompletionActions.jobFailed(exception)); break; } 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 2e36e9e8428d7..9a4a76ab80af5 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 @@ -95,15 +95,13 @@ public class MiniCluster { @GuardedBy("lock") private ResourceManagerRunner[] resourceManagerRunners; - @GuardedBy("lock") - private TaskExecutor[] taskManagers; + private volatile TaskExecutor[] taskManagers; @GuardedBy("lock") private MiniClusterJobDispatcher jobDispatcher; /** Flag marking the mini cluster as started/running */ - @GuardedBy("lock") - private boolean running; + private volatile boolean running; // ------------------------------------------------------------------------ @@ -150,6 +148,8 @@ public MiniCluster(Configuration config) { @Deprecated public MiniCluster(Configuration config, boolean singleRpcService) { this(createConfig(config, singleRpcService)); + + running = false; } // ------------------------------------------------------------------------ @@ -645,17 +645,18 @@ private TerminatingFatalErrorHandler(int index) { @Override public void onFatalError(Throwable exception) { - LOG.error("TaskManager #{} failed.", index, exception); + // first check if we are still running + if (running) { + LOG.error("TaskManager #{} failed.", index, exception); - try { - synchronized (lock) { - // note: if not running (after shutdown) taskManagers may be null! - if (running && taskManagers[index] != null) { - taskManagers[index].shutDown(); - } + // let's check if there are still TaskManagers because there could be a concurrent + // shut down operation taking place + TaskExecutor[] currentTaskManagers = taskManagers; + + if (currentTaskManagers != null) { + // the shutDown is asynchronous + currentTaskManagers[index].shutDown(); } - } catch (Exception e) { - LOG.error("TaskManager #{} could not be properly terminated.", index, e); } } } 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 a9a9e5028f4bc..6b2c8980cf836 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 @@ -35,24 +35,23 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.LeaderIdMismatchException; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; 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.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.registration.JobManagerRegistration; import org.apache.flink.runtime.resourcemanager.registration.WorkerRegistration; -import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.resourcemanager.slotmanager.ResourceManagerActions; +import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerException; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.jobmaster.JobMaster; -import org.apache.flink.runtime.jobmaster.JobMasterGateway; -import org.apache.flink.runtime.registration.RegistrationResponse; - import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutor; @@ -76,7 +75,7 @@ * ResourceManager implementation. The resource manager is responsible for resource de-/allocation * and bookkeeping. * - * It offers the following methods as part of its rpc interface to interact with him remotely: + *

It offers the following methods as part of its rpc interface to interact with him remotely: *

    *
  • {@link #registerJobManager(UUID, UUID, ResourceID, String, JobID, Time)} registers a {@link JobMaster} at the resource manager
  • *
  • {@link #requestSlot(UUID, UUID, SlotRequest, Time)} requests a slot from the resource manager
  • @@ -88,10 +87,10 @@ public abstract class ResourceManager public static final String RESOURCE_MANAGER_NAME = "resourcemanager"; - /** Unique id of the resource manager */ + /** Unique id of the resource manager. */ private final ResourceID resourceId; - /** Configuration of the resource manager */ + /** Configuration of the resource manager. */ private final ResourceManagerConfiguration resourceManagerConfiguration; /** All currently registered JobMasterGateways scoped by JobID. */ @@ -100,7 +99,7 @@ public abstract class ResourceManager /** All currently registered JobMasterGateways scoped by ResourceID. */ private final Map jmResourceIdRegistrations; - /** Service to retrieve the job leader ids */ + /** Service to retrieve the job leader ids. */ private final JobLeaderIdService jobLeaderIdService; /** All currently registered TaskExecutors with there framework specific worker information. */ @@ -115,13 +114,13 @@ public abstract class ResourceManager /** The heartbeat manager with job managers. */ private final HeartbeatManager jobManagerHeartbeatManager; - /** Registry to use for metrics */ + /** Registry to use for metrics. */ private final MetricRegistry metricRegistry; - /** Fatal error handler */ + /** Fatal error handler. */ private final FatalErrorHandler fatalErrorHandler; - /** The slot manager maintains the available slots */ + /** The slot manager maintains the available slots. */ private final SlotManager slotManager; /** The service to elect a ResourceManager leader. */ @@ -268,7 +267,7 @@ public CompletableFuture registerJobManager( ResourceManagerException exception = new ResourceManagerException("Could not add the job " + jobId + " to the job id leader service.", e); - onFatalErrorAsync(exception); + onFatalError(exception); log.error("Could not add job {} to job leader id service.", jobId, e); return FutureUtils.completedExceptionally(exception); @@ -287,7 +286,7 @@ public CompletableFuture registerJobManager( ResourceManagerException exception = new ResourceManagerException("Cannot obtain the " + "job leader id future to verify the correct job leader.", e); - onFatalErrorAsync(exception); + onFatalError(exception); log.debug("Could not obtain the job leader id future to verify the correct job leader."); return FutureUtils.completedExceptionally(exception); @@ -346,7 +345,7 @@ public CompletableFuture registerJobManager( } /** - * Register a {@link TaskExecutor} at the resource manager + * Register a {@link TaskExecutor} at the resource manager. * * @param resourceManagerLeaderId The fencing token for the ResourceManager leader * @param taskExecutorAddress The address of the TaskExecutor that registers @@ -454,7 +453,8 @@ public CompletableFuture requestSlot( } /** - * Notification from a TaskExecutor that a slot has become available + * Notification from a TaskExecutor that a slot has become available. + * * @param resourceManagerLeaderId TaskExecutor's resource manager leader id * @param instanceID TaskExecutor's instance id * @param slotId The slot id of the available slot @@ -491,13 +491,13 @@ public void notifySlotAvailable( } /** - * Registers an info message listener + * Registers an info message listener. * * @param address address of infoMessage listener to register to this resource manager */ @Override public void registerInfoMessageListener(final String address) { - if(infoMessageListeners.containsKey(address)) { + if (infoMessageListeners.containsKey(address)) { log.warn("Receive a duplicate registration from info message listener on ({})", address); } else { CompletableFuture infoMessageListenerRpcGatewayFuture = getRpcService() @@ -517,7 +517,7 @@ public void registerInfoMessageListener(final String address) { } /** - * Unregisters an info message listener + * Unregisters an info message listener. * * @param address of the info message listener to unregister from this resource manager * @@ -528,7 +528,7 @@ public void unRegisterInfoMessageListener(final String address) { } /** - * Cleanup application and shut down cluster + * Cleanup application and shut down cluster. * * @param finalStatus of the Flink application * @param optionalDiagnostics for the Flink application @@ -825,28 +825,15 @@ public void run() { /** * Notifies the ResourceManager that a fatal error has occurred and it cannot proceed. - * This method should be used when asynchronous threads want to notify the - * ResourceManager of a fatal error. - * - * @param t The exception describing the fatal error - */ - protected void onFatalErrorAsync(final Throwable t) { - runAsync(new Runnable() { - @Override - public void run() { - onFatalError(t); - } - }); - } - - /** - * Notifies the ResourceManager that a fatal error has occurred and it cannot proceed. - * This method must only be called from within the ResourceManager's main thread. * * @param t The exception describing the fatal error */ protected void onFatalError(Throwable t) { - log.error("Fatal error occurred.", t); + try { + log.error("Fatal error occurred in ResourceManager.", t); + } catch (Throwable ignored) {} + + // The fatal error handler implementation should make sure that this call is non-blocking fatalErrorHandler.onFatalError(t); } @@ -855,7 +842,7 @@ protected void onFatalError(Throwable t) { // ------------------------------------------------------------------------ /** - * Callback method when current resourceManager is granted leadership + * Callback method when current resourceManager is granted leadership. * * @param newLeaderSessionID unique leadershipID */ @@ -904,13 +891,13 @@ public void run() { } /** - * Handles error occurring in the leader election service + * Handles error occurring in the leader election service. * * @param exception Exception being thrown in the leader election service */ @Override public void handleError(final Exception exception) { - onFatalErrorAsync(new ResourceManagerException("Received an error from the LeaderElectionService.", exception)); + onFatalError(new ResourceManagerException("Received an error from the LeaderElectionService.", exception)); } // ------------------------------------------------------------------------ @@ -928,7 +915,7 @@ public void handleError(final Exception exception) { * The framework specific code for shutting down 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 + *

    This method also needs to make sure all pending containers that are not registered * yet are returned. * * @param finalStatus The application status to report. @@ -1029,7 +1016,7 @@ public void run() { @Override public void handleError(Throwable error) { - onFatalErrorAsync(error); + onFatalError(error); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java index 7721117a240b9..dbccaa87d8732 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/FatalErrorHandler.java @@ -18,7 +18,18 @@ package org.apache.flink.runtime.rpc; +/** + * Handler for fatal errors. + */ public interface FatalErrorHandler { + /** + * Being called when a fatal error occurs. + * + *

    IMPORTANT: This call should never be blocking since it might be called from within + * the main thread of an {@link RpcEndpoint}. + * + * @param exception cause + */ void onFatalError(Throwable exception); } 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 a5ce84be5cdcd..21bdeecc76c00 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 @@ -231,7 +231,7 @@ public void start() throws Exception { try { haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener()); } catch (Exception e) { - onFatalErrorAsync(e); + onFatalError(e); } // tell the task slot table who's responsible for the task slot actions @@ -1117,35 +1117,16 @@ public ResourceID getResourceID() { /** * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed. - * This method should be used when asynchronous threads want to notify the - * TaskExecutor of a fatal error. - * - * @param t The exception describing the fatal error - */ - void onFatalErrorAsync(final Throwable t) { - runAsync(new Runnable() { - @Override - public void run() { - onFatalError(t); - } - }); - } - - /** - * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed. - * This method must only be called from within the TaskExecutor's main thread. * * @param t The exception describing the fatal error */ void onFatalError(final Throwable t) { - log.error("Fatal error occurred.", t); - // this could potentially be a blocking call -> call asynchronously: - getRpcService().execute(new Runnable() { - @Override - public void run() { - fatalErrorHandler.onFatalError(t); - } - }); + try { + log.error("Fatal error occurred in TaskExecutor.", t); + } catch (Throwable ignored) {} + + // The fatal error handler implementation should make sure that this call is non-blocking + fatalErrorHandler.onFatalError(t); } // ------------------------------------------------------------------------ @@ -1183,7 +1164,7 @@ public void run() { @Override public void handleError(Exception exception) { - onFatalErrorAsync(exception); + onFatalError(exception); } } @@ -1223,7 +1204,7 @@ public void run() { @Override public void handleError(Throwable throwable) { - onFatalErrorAsync(throwable); + onFatalError(throwable); } } @@ -1245,7 +1226,7 @@ public void run() { @Override public void onRegistrationFailure(Throwable failure) { - onFatalErrorAsync(failure); + onFatalError(failure); } } @@ -1270,7 +1251,11 @@ public void run() { @Override public void notifyFatalError(String message, Throwable cause) { - log.error(message, cause); + try { + log.error(message, cause); + } catch (Throwable ignored) {} + + // The fatal error handler implementation should make sure that this call is non-blocking fatalErrorHandler.onFatalError(cause); } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java index 00b73a863ce1f..3e58da5ae5890 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptorV2.java @@ -18,9 +18,7 @@ package org.apache.flink.yarn; -import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; @@ -45,9 +43,4 @@ protected String getYarnSessionClusterEntrypoint() { protected String getYarnJobClusterEntrypoint() { return YarnJobClusterEntrypoint.class.getName(); } - - @Override - public YarnClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph) { - throw new UnsupportedOperationException("Cannot yet deploy a per-job yarn cluster."); - } } 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 c3398c44e0603..dd12fefaedc00 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 @@ -293,7 +293,7 @@ public void onNodesUpdated(List list) { @Override public void onError(Throwable error) { - onFatalErrorAsync(error); + onFatalError(error); } //Utility methods From a1578710a8aa4c2cc862ae9ed3cf807133273d67 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 24 Aug 2017 13:24:15 +0200 Subject: [PATCH 085/129] [FLINK-7500] Set JobMaster leader session id in main thread This commit changes the JobMaster such that it's leader session id is only set in the main thread. This is done by passing the leader session id to the startJobExecution method. Moreover, this commit returns a future from the start and suspend methods which can be used to wait on the completion of these operations. This closes #4579. --- .../runtime/jobmaster/JobManagerRunner.java | 27 +++++- .../flink/runtime/jobmaster/JobMaster.java | 88 ++++++++++--------- .../jobmaster/JobManagerRunnerMockTest.java | 17 ++-- .../runtime/jobmaster/JobMasterTest.java | 15 +++- 4 files changed, 91 insertions(+), 56 deletions(-) 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 8f1be4c7a1901..8766fabb7a21e 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 @@ -20,6 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.client.JobExecutionException; @@ -33,6 +34,7 @@ import org.apache.flink.runtime.jobmanager.OnCompletionActions; 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.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; @@ -44,6 +46,7 @@ import java.io.IOException; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -82,6 +85,8 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F private final JobManagerMetricGroup jobManagerMetricGroup; + private final Time timeout; + /** flag marking the runner as shut down */ private volatile boolean shutdown; @@ -207,6 +212,8 @@ public JobManagerRunner( this, this, userCodeLoader); + + this.timeout = jobManagerServices.rpcAskTimeout; } catch (Throwable t) { // clean up everything @@ -415,7 +422,15 @@ public void grantLeadership(final UUID leaderSessionID) { runningJobsRegistry.setJobRunning(jobGraph.getJobID()); } - jobManager.start(leaderSessionID); + CompletableFuture startingFuture = jobManager.start(leaderSessionID, timeout); + + startingFuture.whenCompleteAsync( + (Acknowledge ack, Throwable throwable) -> { + if (throwable != null) { + onFatalError(new Exception("Could not start the job manager.", throwable)); + } + }, + jobManagerServices.executorService); } catch (Exception e) { onFatalError(new Exception("Could not start the job manager.", e)); } @@ -434,7 +449,15 @@ public void revokeLeadership() { log.info("JobManager for job {} ({}) was revoked leadership at {}.", jobGraph.getName(), jobGraph.getJobID(), getAddress()); - jobManager.suspend(new Exception("JobManager is no longer the leader.")); + CompletableFuture suspendFuture = jobManager.suspend(new Exception("JobManager is no longer the leader."), timeout); + + suspendFuture.whenCompleteAsync( + (Acknowledge ack, Throwable throwable) -> { + if (throwable != null) { + onFatalError(new Exception("Could not start the job manager.", throwable)); + } + }, + jobManagerServices.executorService); } } 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 3e66a3419bf60..55bc8f8677b92 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 @@ -90,6 +90,7 @@ import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedThrowable; @@ -109,7 +110,6 @@ import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -130,9 +130,6 @@ public class JobMaster extends RpcEndpoint implements JobMasterGateway { public static final String JOB_MANAGER_NAME = "jobmanager"; public static final String ARCHIVE_NAME = "archive"; - private static final AtomicReferenceFieldUpdater LEADER_ID_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(JobMaster.class, UUID.class, "leaderSessionID"); - // ------------------------------------------------------------------------ private final JobMasterGateway selfGateway; @@ -311,18 +308,14 @@ public void start() { * Start the rpc service and begin to run the job. * * @param leaderSessionID The necessary leader id for running the job. + * @param timeout for the operation + * @return Future acknowledge if the job could be started. Otherwise the future contains an exception */ - public void start(final UUID leaderSessionID) throws Exception { - if (LEADER_ID_UPDATER.compareAndSet(this, null, leaderSessionID)) { - // make sure we receive RPC and async calls - super.start(); + public CompletableFuture start(final UUID leaderSessionID, final Time timeout) throws Exception { + // make sure we receive RPC and async calls + super.start(); - log.info("JobManager started as leader {} for job {}", leaderSessionID, jobGraph.getJobID()); - runAsync(this::startJobExecution); - } - else { - log.warn("Job already started with leader ID {}, ignoring this start request.", leaderSessionID); - } + return callAsync(() -> startJobExecution(leaderSessionID), timeout); } /** @@ -330,14 +323,20 @@ public void start(final UUID leaderSessionID) throws Exception { * will be disposed. * *

    Mostly job is suspended because of the leadership has been revoked, one can be restart this job by - * calling the {@link #start(UUID)} method once we take the leadership back again. + * calling the {@link #start(UUID, Time)} method once we take the leadership back again. * *

    This method is executed asynchronously * * @param cause The reason of why this job been suspended. + * @param timeout for this operation + * @return Future acknowledge indicating that the job has been suspended. Otherwise the future contains an exception */ - public void suspend(final Throwable cause) { - runAsync(() -> suspendExecution(cause)); + public CompletableFuture suspend(final Throwable cause, final Time timeout) { + CompletableFuture suspendFuture = callAsync(() -> suspendExecution(cause), timeout); + + stop(); + + return suspendFuture; } /** @@ -788,11 +787,26 @@ public void heartbeatFromResourceManager(final ResourceID resourceID) { //-- job starting and stopping ----------------------------------------------------------------- - private void startJobExecution() { - // double check that the leader status did not change + private Acknowledge startJobExecution(UUID newLeaderSessionId) throws Exception { + Preconditions.checkNotNull(newLeaderSessionId, "The new leader session id must not be null."); + if (leaderSessionID == null) { - log.info("Aborting job startup - JobManager lost leader status"); - return; + log.info("Start job execution with leader id {}.", newLeaderSessionId); + + leaderSessionID = newLeaderSessionId; + } else if (Objects.equals(leaderSessionID, newLeaderSessionId)) { + log.info("Already started the job execution with leader id {}.", leaderSessionID); + + return Acknowledge.get(); + } else { + log.info("Restarting old job with leader id {}. The new leader id is {}.", leaderSessionID, newLeaderSessionId); + + // first we have to suspend the current execution + suspendExecution(new FlinkException("Old job with leader id " + leaderSessionID + + " is restarted with a new leader id " + newLeaderSessionId + '.')); + + // set new leader id + leaderSessionID = newLeaderSessionId; } log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID()); @@ -801,13 +815,7 @@ private void startJobExecution() { // start the slot pool make sure the slot pool now accepts messages for this leader log.debug("Staring SlotPool component"); slotPool.start(leaderSessionID, getAddress()); - } catch (Exception e) { - log.error("Faild to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), e); - - handleFatalError(new Exception("Could not start job execution: Failed to start the slot pool.", e)); - } - try { // job is ready to go, try to establish connection with resource manager // - activate leader retrieval for the resource manager // - on notification of the leader, the connection will be established and @@ -817,24 +825,21 @@ private void startJobExecution() { catch (Throwable t) { log.error("Failed to start job {} ({})", jobGraph.getName(), jobGraph.getJobID(), t); - handleFatalError(new Exception( - "Could not start job execution: Failed to start leader service for Resource Manager", t)); - - return; + throw new Exception("Could not start job execution: Failed to start JobMaster services.", t); } // start scheduling job in another thread - executor.execute(new Runnable() { - @Override - public void run() { + executor.execute( + () -> { try { executionGraph.scheduleForExecution(); } catch (Throwable t) { executionGraph.failGlobal(t); } - } - }); + }); + + return Acknowledge.get(); } /** @@ -842,14 +847,14 @@ public void run() { * will be disposed. * *

    Mostly job is suspended because of the leadership has been revoked, one can be restart this job by - * calling the {@link #start(UUID)} method once we take the leadership back again. + * calling the {@link #start(UUID, Time)} method once we take the leadership back again. * * @param cause The reason of why this job been suspended. */ - private void suspendExecution(final Throwable cause) { + private Acknowledge suspendExecution(final Throwable cause) { if (leaderSessionID == null) { log.debug("Job has already been suspended or shutdown."); - return; + return Acknowledge.get(); } // not leader any more - should not accept any leader messages any more @@ -864,14 +869,13 @@ private void suspendExecution(final Throwable cause) { // tell the execution graph (JobManager is still processing messages here) executionGraph.suspend(cause); - // receive no more messages until started again, should be called before we clear self leader id - stop(); - // the slot pool stops receiving messages and clears its pooled slots slotPoolGateway.suspend(); // disconnect from resource manager: closeResourceManagerConnection(new Exception("Execution was suspended.", cause)); + + return Acknowledge.get(); } //---------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java index 998e803b83148..0f38db2f1c890 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.BlobStore; @@ -132,7 +133,7 @@ public void testStartAndShutdown() throws Exception { assertTrue(!jobCompletion.isJobFinished()); assertTrue(!jobCompletion.isJobFailed()); - verify(jobManager).start(any(UUID.class)); + verify(jobManager).start(any(UUID.class), any(Time.class)); runner.shutdown(); verify(leaderElectionService).stop(); @@ -166,7 +167,7 @@ public void testJobFinished() throws Exception { UUID leaderSessionID = UUID.randomUUID(); runner.grantLeadership(leaderSessionID); - verify(jobManager).start(leaderSessionID); + verify(jobManager).start(eq(leaderSessionID), any(Time.class)); assertTrue(!jobCompletion.isJobFinished()); // runner been told by JobManager that job is finished @@ -186,7 +187,7 @@ public void testJobFailed() throws Exception { UUID leaderSessionID = UUID.randomUUID(); runner.grantLeadership(leaderSessionID); - verify(jobManager).start(leaderSessionID); + verify(jobManager).start(eq(leaderSessionID), any(Time.class)); assertTrue(!jobCompletion.isJobFinished()); // runner been told by JobManager that job is failed @@ -205,11 +206,11 @@ public void testLeadershipRevoked() throws Exception { UUID leaderSessionID = UUID.randomUUID(); runner.grantLeadership(leaderSessionID); - verify(jobManager).start(leaderSessionID); + verify(jobManager).start(eq(leaderSessionID), any(Time.class)); assertTrue(!jobCompletion.isJobFinished()); runner.revokeLeadership(); - verify(jobManager).suspend(any(Throwable.class)); + verify(jobManager).suspend(any(Throwable.class), any(Time.class)); assertFalse(runner.isShutdown()); } @@ -220,16 +221,16 @@ public void testRegainLeadership() throws Exception { UUID leaderSessionID = UUID.randomUUID(); runner.grantLeadership(leaderSessionID); - verify(jobManager).start(leaderSessionID); + verify(jobManager).start(eq(leaderSessionID), any(Time.class)); assertTrue(!jobCompletion.isJobFinished()); runner.revokeLeadership(); - verify(jobManager).suspend(any(Throwable.class)); + verify(jobManager).suspend(any(Throwable.class), any(Time.class)); assertFalse(runner.isShutdown()); UUID leaderSessionID2 = UUID.randomUUID(); runner.grantLeadership(leaderSessionID2); - verify(jobManager).start(leaderSessionID2); + verify(jobManager).start(eq(leaderSessionID2), any(Time.class)); } private static class TestingOnCompletionActions implements OnCompletionActions, FatalErrorHandler { 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 2c17b5a78587e..76a0b93104c03 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 @@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.TestingRpcService; @@ -110,13 +111,16 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { blobServer, mock(BlobLibraryCacheManager.class), mock(RestartStrategyFactory.class), - Time.of(10, TimeUnit.SECONDS), + testingTimeout, null, mock(OnCompletionActions.class), testingFatalErrorHandler, new FlinkUserCodeClassLoader(new URL[0])); - jobMaster.start(jmLeaderId); + CompletableFuture startFuture = jobMaster.start(jmLeaderId, testingTimeout); + + // wait for the start to complete + startFuture.get(testingTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); final JobMasterGateway jobMasterGateway = jobMaster.getSelfGateway(JobMasterGateway.class); @@ -209,13 +213,16 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { mock(BlobServer.class), mock(BlobLibraryCacheManager.class), mock(RestartStrategyFactory.class), - Time.of(10, TimeUnit.SECONDS), + testingTimeout, null, mock(OnCompletionActions.class), testingFatalErrorHandler, new FlinkUserCodeClassLoader(new URL[0])); - jobMaster.start(jmLeaderId); + CompletableFuture startFuture = jobMaster.start(jmLeaderId, testingTimeout); + + // wait for the start operation to complete + startFuture.get(testingTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); // define a leader and see that a registration happens rmLeaderRetrievalService.notifyListener(resourceManagerAddress, rmLeaderId); From 64e8de97d02a747da52a3e69472147b64fd2dd91 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Aug 2017 12:40:22 +0200 Subject: [PATCH 086/129] [FLINK-7526] [TaskExecutor] Filter out duplicate JobManager gained leadership messages This commit filters out duplicate JobManager gained leadership messges coming from the JobLeaderService. This avoid opening multiple connections to the JobManager which consumes resources. Moreover, this commit properly closes all JobManagerConnections in case of a shut down. This closes #4595. --- .../runtime/taskexecutor/TaskExecutor.java | 29 +- .../runtime/taskexecutor/slot/TaskSlot.java | 6 + .../taskexecutor/slot/TaskSlotTable.java | 12 +- .../taskexecutor/TaskExecutorTest.java | 696 ++++++++++-------- 4 files changed, 429 insertions(+), 314 deletions(-) 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 21bdeecc76c00..ef47ad4c6a49b 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 @@ -85,6 +85,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import java.io.IOException; @@ -248,7 +249,7 @@ public void start() throws Exception { public void postStop() throws Exception { log.info("Stopping TaskManager {}.", getAddress()); - Exception exception = null; + Throwable throwable = null; taskSlotTable.stop(); @@ -256,6 +257,14 @@ public void postStop() throws Exception { resourceManagerConnection.close(); } + for (JobManagerConnection jobManagerConnection : jobManagerConnections.values()) { + try { + disassociateFromJobManager(jobManagerConnection, new FlinkException("The TaskExecutor is shutting down.")); + } catch (Throwable t) { + throwable = ExceptionUtils.firstOrSuppressed(t, throwable); + } + } + jobManagerHeartbeatManager.stop(); resourceManagerHeartbeatManager.stop(); @@ -270,12 +279,12 @@ public void postStop() throws Exception { try { super.postStop(); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); + } catch (Throwable e) { + throwable = ExceptionUtils.firstOrSuppressed(e, throwable); } - if (exception != null) { - ExceptionUtils.rethrowException(exception, "Error while shutting the TaskExecutor down."); + if (throwable != null) { + ExceptionUtils.rethrowException(throwable, "Error while shutting the TaskExecutor down."); } log.info("Stopped TaskManager {}.", getAddress()); @@ -841,15 +850,21 @@ private void offerSlotsToJobManager(final JobID jobId) { } private void establishJobManagerConnection(JobID jobId, final JobMasterGateway jobMasterGateway, UUID jobManagerLeaderId, JMTMRegistrationSuccess registrationSuccess) { - log.info("Establish JobManager connection for job {}.", jobId); if (jobManagerTable.contains(jobId)) { JobManagerConnection oldJobManagerConnection = jobManagerTable.get(jobId); - if (!oldJobManagerConnection.getLeaderId().equals(jobManagerLeaderId)) { + + if (Objects.equals(oldJobManagerConnection.getLeaderId(), jobManagerLeaderId)) { + // we already are connected to the given job manager + log.debug("Ignore JobManager gained leadership message for {} because we are already connected to it.", jobManagerLeaderId); + return; + } else { closeJobManagerConnection(jobId, new Exception("Found new job leader for job id " + jobId + '.')); } } + log.info("Establish JobManager connection for job {}.", jobId); + ResourceID jobManagerResourceID = registrationSuccess.getResourceID(); JobManagerConnection newJobManagerConnection = associateWithJobManager( jobId, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java index e12c15b5c411c..6f5230cec0206 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlot.java @@ -299,4 +299,10 @@ public SlotOffer generateSlotOffer() { return new SlotOffer(allocationId, index, resourceProfile); } + + @Override + public String toString() { + return "TaskSlot(index:" + index + ", state:" + state + ", resource profile: " + resourceProfile + + ", allocationId: " + (allocationId != null ? allocationId.toString() : "none") + ", jobId: " + (jobId != null ? jobId.toString() : "none") + ')'; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java index 3634df00d88e9..799f639cb5615 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTable.java @@ -282,15 +282,15 @@ public int freeSlot(AllocationID allocationId) throws SlotNotFoundException { public int freeSlot(AllocationID allocationId, Throwable cause) throws SlotNotFoundException { checkInit(); - if (LOG.isDebugEnabled()) { - LOG.debug("Free slot {}.", allocationId, cause); - } else { - LOG.info("Free slot {}.", allocationId); - } - TaskSlot taskSlot = getTaskSlot(allocationId); if (taskSlot != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Free slot {}.", taskSlot, cause); + } else { + LOG.info("Free slot {}.", taskSlot); + } + final JobID jobId = taskSlot.getJobId(); if (taskSlot.markFree()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 41e56294c6d3e..6ab52ed46e4fc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -65,6 +65,7 @@ import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; @@ -72,12 +73,16 @@ import org.apache.flink.runtime.taskexecutor.slot.TimerService; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.Task; +import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -97,6 +102,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import static org.hamcrest.Matchers.contains; @@ -112,6 +118,21 @@ public class TaskExecutorTest extends TestLogger { private final Time timeout = Time.milliseconds(10000L); + private TestingRpcService rpc; + + @Before + public void setup() { + rpc = new TestingRpcService(); + } + + @After + public void teardown() { + if (rpc != null) { + rpc.stopService(); + rpc = null; + } + } + @Rule public TestName name = new TestName(); @@ -124,7 +145,6 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(tmResourceId, InetAddress.getLoopbackAddress(), 1234); final TaskSlotTable taskSlotTable = new TaskSlotTable(Arrays.asList(mock(ResourceProfile.class)), mock(TimerService.class)); - final TestingRpcService rpc = new TestingRpcService(); final JobLeaderService jobLeaderService = new JobLeaderService(taskManagerLocation); final TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); final TestingLeaderRetrievalService rmLeaderRetrievalService = new TestingLeaderRetrievalService( @@ -175,25 +195,25 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro when(jobMasterGateway.getAddress()).thenReturn(jobMasterAddress); when(jobMasterGateway.getHostname()).thenReturn("localhost"); - try { - final TaskExecutor taskManager = new TaskExecutor( - rpc, - tmConfig, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - mock(NetworkEnvironment.class), - haServices, - heartbeatServices, - mock(MetricRegistry.class), - mock(TaskManagerMetricGroup.class), - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - new JobManagerTable(), - jobLeaderService, - testingFatalErrorHandler); + final TaskExecutor taskManager = new TaskExecutor( + rpc, + tmConfig, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices, + heartbeatServices, + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + new JobManagerTable(), + jobLeaderService, + testingFatalErrorHandler); + try { taskManager.start(); rpc.registerGateway(jobMasterAddress, jobMasterGateway); @@ -216,7 +236,8 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -239,7 +260,6 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { rmResourceId, 10L))); - final TestingRpcService rpc = new TestingRpcService(); rpc.registerGateway(rmAddress, rmGateway); final TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService( @@ -281,25 +301,25 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation } ); - try { - final TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerConfiguration, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - mock(NetworkEnvironment.class), - haServices, - heartbeatServices, - mock(MetricRegistry.class), - mock(TaskManagerMetricGroup.class), - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - mock(JobManagerTable.class), - mock(JobLeaderService.class), - testingFatalErrorHandler); + final TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices, + heartbeatServices, + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + mock(JobManagerTable.class), + mock(JobLeaderService.class), + testingFatalErrorHandler); + try { taskManager.start(); // define a leader and see that a registration happens @@ -316,7 +336,8 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -325,7 +346,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation */ @Test public void testHeartbeatSlotReporting() throws Exception { - final long timeout = 1000L; + final long verificationTimeout = 1000L; final String rmAddress = "rm"; final String tmAddress = "tm"; final ResourceID rmResourceId = new ResourceID(rmAddress); @@ -343,7 +364,6 @@ public void testHeartbeatSlotReporting() throws Exception { rmResourceId, 10L))); - final TestingRpcService rpc = new TestingRpcService(); rpc.registerGateway(rmAddress, rmGateway); final TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService( @@ -399,25 +419,25 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation } ); - try { - final TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerConfiguration, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - mock(NetworkEnvironment.class), - haServices, - heartbeatServices, - mock(MetricRegistry.class), - mock(TaskManagerMetricGroup.class), - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - mock(JobManagerTable.class), - mock(JobLeaderService.class), - testingFatalErrorHandler); + final TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices, + heartbeatServices, + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + mock(JobManagerTable.class), + mock(JobLeaderService.class), + testingFatalErrorHandler); + try { taskManager.start(); // wait for spied heartbeat manager instance @@ -427,10 +447,10 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation testLeaderService.notifyListener(rmAddress, rmLeaderId); // register resource manager success will trigger monitoring heartbeat target between tm and rm - verify(rmGateway, timeout(timeout).atLeast(1)).registerTaskExecutor( + verify(rmGateway, timeout(verificationTimeout).atLeast(1)).registerTaskExecutor( eq(rmLeaderId), eq(taskManager.getAddress()), eq(tmResourceId), eq(slotReport1), any(Time.class)); - verify(heartbeatManager, timeout(timeout)).monitorTarget(any(ResourceID.class), any(HeartbeatTarget.class)); + verify(heartbeatManager, timeout(verificationTimeout)).monitorTarget(any(ResourceID.class), any(HeartbeatTarget.class)); TaskExecutorGateway taskExecutorGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); @@ -440,7 +460,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation ArgumentCaptor slotReportArgumentCaptor = ArgumentCaptor.forClass(SlotReport.class); // wait for heartbeat response - verify(rmGateway, timeout(timeout)).heartbeatFromTaskManager( + verify(rmGateway, timeout(verificationTimeout)).heartbeatFromTaskManager( eq(taskManagerLocation.getResourceID()), slotReportArgumentCaptor.capture()); @@ -453,7 +473,8 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -465,52 +486,51 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { final String dispatcherAddress = "localhost"; final String jobManagerAddress = "localhost"; - final TestingRpcService rpc = new TestingRpcService(); - try { - // register a mock resource manager gateway - ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); - when(rmGateway.registerTaskExecutor( - any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess( - new InstanceID(), resourceManagerResourceId, 10L))); + // register a mock resource manager gateway + ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); + when(rmGateway.registerTaskExecutor( + any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) + .thenReturn(CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess( + new InstanceID(), resourceManagerResourceId, 10L))); - TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); - when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); + TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); + when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); - rpc.registerGateway(resourceManagerAddress, rmGateway); + rpc.registerGateway(resourceManagerAddress, rmGateway); - TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); - when(taskManagerLocation.getResourceID()).thenReturn(resourceID); + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); + when(taskManagerLocation.getResourceID()).thenReturn(resourceID); - StandaloneHaServices haServices = new StandaloneHaServices( - resourceManagerAddress, + StandaloneHaServices haServices = new StandaloneHaServices( + resourceManagerAddress, dispatcherAddress, - jobManagerAddress); - - final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class); - final SlotReport slotReport = new SlotReport(); - when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport); - - final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - - TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerServicesConfiguration, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - mock(NetworkEnvironment.class), - haServices, - mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), - mock(TaskManagerMetricGroup.class), - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - mock(JobManagerTable.class), - mock(JobLeaderService.class), - testingFatalErrorHandler); + jobManagerAddress); + + final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class); + final SlotReport slotReport = new SlotReport(); + when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport); + final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + + TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerServicesConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices, + mock(HeartbeatServices.class, RETURNS_MOCKS), + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + mock(JobManagerTable.class), + mock(JobLeaderService.class), + testingFatalErrorHandler); + + try { taskManager.start(); String taskManagerAddress = taskManager.getAddress(); @@ -521,7 +541,8 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -536,64 +557,63 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { final ResourceID rmResourceId1 = new ResourceID(address1); final ResourceID rmResourceId2 = new ResourceID(address2); - final TestingRpcService rpc = new TestingRpcService(); - try { - // register the mock resource manager gateways - ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class); - ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class); - - when(rmGateway1.registerTaskExecutor( - any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(CompletableFuture.completedFuture( - new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId1, 10L))); - when(rmGateway2.registerTaskExecutor( - any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) - .thenReturn(CompletableFuture.completedFuture( - new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId2, 10L))); - - rpc.registerGateway(address1, rmGateway1); - rpc.registerGateway(address2, rmGateway2); - - TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService( - null, - null); - - TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); - haServices.setResourceManagerLeaderRetriever(testLeaderService); - - TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); - when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); - when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration()); - when(taskManagerServicesConfiguration.getTmpDirectories()).thenReturn(new String[1]); - - TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); - when(taskManagerLocation.getResourceID()).thenReturn(tmResourceID); - when(taskManagerLocation.getHostname()).thenReturn("foobar"); - - final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class); - final SlotReport slotReport = new SlotReport(); - when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport); - - final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - - TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerServicesConfiguration, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - mock(NetworkEnvironment.class), - haServices, - mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), - mock(TaskManagerMetricGroup.class), - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - mock(JobManagerTable.class), - mock(JobLeaderService.class), - testingFatalErrorHandler); + // register the mock resource manager gateways + ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class); + ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class); + + when(rmGateway1.registerTaskExecutor( + any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) + .thenReturn(CompletableFuture.completedFuture( + new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId1, 10L))); + when(rmGateway2.registerTaskExecutor( + any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) + .thenReturn(CompletableFuture.completedFuture( + new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId2, 10L))); + rpc.registerGateway(address1, rmGateway1); + rpc.registerGateway(address2, rmGateway2); + + TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService( + null, + null); + + TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); + haServices.setResourceManagerLeaderRetriever(testLeaderService); + + TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); + when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); + when(taskManagerServicesConfiguration.getConfiguration()).thenReturn(new Configuration()); + when(taskManagerServicesConfiguration.getTmpDirectories()).thenReturn(new String[1]); + + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); + when(taskManagerLocation.getResourceID()).thenReturn(tmResourceID); + when(taskManagerLocation.getHostname()).thenReturn("foobar"); + + final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class); + final SlotReport slotReport = new SlotReport(); + when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(slotReport); + + final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + + TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerServicesConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices, + mock(HeartbeatServices.class, RETURNS_MOCKS), + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + mock(JobManagerTable.class), + mock(JobLeaderService.class), + testingFatalErrorHandler); + + try { taskManager.start(); String taskManagerAddress = taskManager.getAddress(); @@ -621,7 +641,8 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -632,7 +653,6 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { public void testTaskSubmission() throws Exception { final Configuration configuration = new Configuration(); - final TestingRpcService rpc = new TestingRpcService(); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final JobID jobId = new JobID(); final AllocationID allocationId = new AllocationID(); @@ -709,27 +729,27 @@ public void testTaskSubmission() throws Exception { final HighAvailabilityServices haServices = mock(HighAvailabilityServices.class); when(haServices.getResourceManagerLeaderRetriever()).thenReturn(mock(LeaderRetrievalService.class)); - try { - final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - - TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerConfiguration, - mock(TaskManagerLocation.class), - mock(MemoryManager.class), - mock(IOManager.class), - networkEnvironment, - haServices, - mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), - taskManagerMetricGroup, - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - jobManagerTable, - mock(JobLeaderService.class), - testingFatalErrorHandler); + final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + + TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerConfiguration, + mock(TaskManagerLocation.class), + mock(MemoryManager.class), + mock(IOManager.class), + networkEnvironment, + haServices, + mock(HeartbeatServices.class, RETURNS_MOCKS), + mock(MetricRegistry.class), + taskManagerMetricGroup, + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + jobManagerTable, + mock(JobLeaderService.class), + testingFatalErrorHandler); + try { taskManager.start(); final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); @@ -743,7 +763,8 @@ public void testTaskSubmission() throws Exception { // check if a concurrent error occurred testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -768,7 +789,6 @@ public void invoke() throws Exception { public void testJobLeaderDetection() throws Exception { final JobID jobId = new JobID(); - final TestingRpcService rpc = new TestingRpcService(); final Configuration configuration = new Configuration(); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final ResourceID resourceId = new ResourceID("foobar"); @@ -830,25 +850,25 @@ public void testJobLeaderDetection() throws Exception { final SlotID slotId = new SlotID(resourceId, 0); final SlotOffer slotOffer = new SlotOffer(allocationId, 0, ResourceProfile.UNKNOWN); - try { - TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerConfiguration, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - mock(NetworkEnvironment.class), - haServices, - mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), - mock(TaskManagerMetricGroup.class), - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - jobManagerTable, - jobLeaderService, - testingFatalErrorHandler); + TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices, + mock(HeartbeatServices.class, RETURNS_MOCKS), + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + jobManagerTable, + jobLeaderService, + testingFatalErrorHandler); + try { taskManager.start(); final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); @@ -880,7 +900,8 @@ public void testJobLeaderDetection() throws Exception { // check if a concurrent error occurred testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -892,7 +913,6 @@ public void testJobLeaderDetection() throws Exception { public void testSlotAcceptance() throws Exception { final JobID jobId = new JobID(); - final TestingRpcService rpc = new TestingRpcService(); final Configuration configuration = new Configuration(); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final ResourceID resourceId = new ResourceID("foobar"); @@ -951,25 +971,25 @@ public void testSlotAcceptance() throws Exception { rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); rpc.registerGateway(jobManagerAddress, jobMasterGateway); - try { - TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerConfiguration, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - mock(NetworkEnvironment.class), - haServices, - mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), - mock(TaskManagerMetricGroup.class), - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - jobManagerTable, - jobLeaderService, - testingFatalErrorHandler); + TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices, + mock(HeartbeatServices.class, RETURNS_MOCKS), + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + jobManagerTable, + jobLeaderService, + testingFatalErrorHandler); + try { taskManager.start(); taskSlotTable.allocateSlot(0, jobId, allocationId1, Time.milliseconds(10000L)); @@ -992,7 +1012,8 @@ public void testSlotAcceptance() throws Exception { // check if a concurrent error occurred testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -1012,51 +1033,50 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { final JobID jobId = new JobID(); final String jobManagerAddress = "foobar"; - final TestingRpcService rpc = new TestingRpcService(); - try { - // register the mock resource manager gateways - ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class); - rpc.registerGateway(address1, rmGateway1); - - TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService( - address1, - HighAvailabilityServices.DEFAULT_LEADER_ID); - - TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); - haServices.setResourceManagerLeaderRetriever(testLeaderService); - - TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); - when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); - - TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); - when(taskManagerLocation.getResourceID()).thenReturn(resourceID); - - final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class); - when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(new SlotReport()); - when(taskSlotTable.getCurrentAllocation(1)).thenReturn(new AllocationID()); - - when(rmGateway1.registerTaskExecutor(any(UUID.class), anyString(), eq(resourceID), any(SlotReport.class), any(Time.class))).thenReturn( - CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(new InstanceID(), ResourceID.generate(), 1000L))); - - TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerServicesConfiguration, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - mock(NetworkEnvironment.class), - haServices, - mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), - mock(TaskManagerMetricGroup.class), - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - mock(JobManagerTable.class), - mock(JobLeaderService.class), - testingFatalErrorHandler); + // register the mock resource manager gateways + ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class); + rpc.registerGateway(address1, rmGateway1); + TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService( + address1, + HighAvailabilityServices.DEFAULT_LEADER_ID); + + TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices(); + haServices.setResourceManagerLeaderRetriever(testLeaderService); + + TaskManagerConfiguration taskManagerServicesConfiguration = mock(TaskManagerConfiguration.class); + when(taskManagerServicesConfiguration.getNumberSlots()).thenReturn(1); + + TaskManagerLocation taskManagerLocation = mock(TaskManagerLocation.class); + when(taskManagerLocation.getResourceID()).thenReturn(resourceID); + + final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + final TaskSlotTable taskSlotTable = mock(TaskSlotTable.class); + when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(new SlotReport()); + when(taskSlotTable.getCurrentAllocation(1)).thenReturn(new AllocationID()); + + when(rmGateway1.registerTaskExecutor(any(UUID.class), anyString(), eq(resourceID), any(SlotReport.class), any(Time.class))).thenReturn( + CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(new InstanceID(), ResourceID.generate(), 1000L))); + + TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerServicesConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServices, + mock(HeartbeatServices.class, RETURNS_MOCKS), + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + mock(JobManagerTable.class), + mock(JobLeaderService.class), + testingFatalErrorHandler); + + try { taskManager.start(); final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); @@ -1116,7 +1136,8 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } } @@ -1128,7 +1149,6 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { public void testSubmitTaskBeforeAcceptSlot() throws Exception { final JobID jobId = new JobID(); - final TestingRpcService rpc = new TestingRpcService(); final Configuration configuration = new Configuration(); final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final ResourceID resourceId = new ResourceID("foobar"); @@ -1180,6 +1200,9 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { any(Time.class) )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); + when(jobMasterGateway.updateTaskExecutionState( + any(UUID.class), + any(TaskExecutionState.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); @@ -1200,37 +1223,36 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { mock(ResultPartitionConsumableNotifier.class), mock(PartitionProducerStateChecker.class)); - jobManagerTable.put(jobId, jobManagerConnection); - - try { - final TaskManagerMetricGroup taskManagerMetricGroup = mock(TaskManagerMetricGroup.class); - TaskMetricGroup taskMetricGroup = mock(TaskMetricGroup.class); - when(taskMetricGroup.getIOMetricGroup()).thenReturn(mock(TaskIOMetricGroup.class)); + final TaskManagerMetricGroup taskManagerMetricGroup = mock(TaskManagerMetricGroup.class); + TaskMetricGroup taskMetricGroup = mock(TaskMetricGroup.class); + when(taskMetricGroup.getIOMetricGroup()).thenReturn(mock(TaskIOMetricGroup.class)); - when(taskManagerMetricGroup.addTaskForJob( - any(JobID.class), anyString(), any(JobVertexID.class), any(ExecutionAttemptID.class), - anyString(), anyInt(), anyInt()) - ).thenReturn(taskMetricGroup); + when(taskManagerMetricGroup.addTaskForJob( + any(JobID.class), anyString(), any(JobVertexID.class), any(ExecutionAttemptID.class), + anyString(), anyInt(), anyInt()) + ).thenReturn(taskMetricGroup); + + final NetworkEnvironment networkMock = mock(NetworkEnvironment.class, Mockito.RETURNS_MOCKS); + + final TaskExecutor taskManager = new TaskExecutor( + rpc, + taskManagerConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + networkMock, + haServices, + mock(HeartbeatServices.class, RETURNS_MOCKS), + mock(MetricRegistry.class), + taskManagerMetricGroup, + mock(BroadcastVariableManager.class), + mock(FileCache.class), + taskSlotTable, + jobManagerTable, + jobLeaderService, + testingFatalErrorHandler); - final NetworkEnvironment networkMock = mock(NetworkEnvironment.class, Mockito.RETURNS_MOCKS); - - final TaskExecutor taskManager = new TaskExecutor( - rpc, - taskManagerConfiguration, - taskManagerLocation, - mock(MemoryManager.class), - mock(IOManager.class), - networkMock, - haServices, - mock(HeartbeatServices.class, RETURNS_MOCKS), - mock(MetricRegistry.class), - taskManagerMetricGroup, - mock(BroadcastVariableManager.class), - mock(FileCache.class), - taskSlotTable, - jobManagerTable, - jobLeaderService, - testingFatalErrorHandler); + try { taskManager.start(); final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); @@ -1307,8 +1329,80 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { // check if a concurrent error occurred testingFatalErrorHandler.rethrowError(); } finally { - rpc.stopService(); + taskManager.shutDown(); + taskManager.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } + } + + /** + * This tests makes sure that duplicate JobMaster gained leadership messages are filtered out + * by the TaskExecutor. + * + * See FLINK-7526 + */ + @Test + public void testFilterOutDuplicateJobMasterRegistrations() throws Exception { + final long verificationTimeout = 500L; + final Configuration configuration = new Configuration(); + final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); + final JobLeaderService jobLeaderService = mock(JobLeaderService.class); + final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); + final TaskManagerLocation taskManagerLocation = new TaskManagerLocation(ResourceID.generate(), InetAddress.getLocalHost(), 1234); + + final HighAvailabilityServices haServicesMock = mock(HighAvailabilityServices.class, Mockito.RETURNS_MOCKS); + final HeartbeatServices heartbeatServicesMock = mock(HeartbeatServices.class, Mockito.RETURNS_MOCKS); + + final JobID jobId = new JobID(); + final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); + when(jobMasterGateway.getHostname()).thenReturn("localhost"); + final UUID jobLeaderId = UUID.randomUUID(); + final JMTMRegistrationSuccess registrationMessage = new JMTMRegistrationSuccess(ResourceID.generate(), 1); + final JobManagerTable jobManagerTableMock = spy(new JobManagerTable()); + + final TaskExecutor taskExecutor = new TaskExecutor( + rpc, + taskManagerConfiguration, + taskManagerLocation, + mock(MemoryManager.class), + mock(IOManager.class), + mock(NetworkEnvironment.class), + haServicesMock, + heartbeatServicesMock, + mock(MetricRegistry.class), + mock(TaskManagerMetricGroup.class), + mock(BroadcastVariableManager.class), + mock(FileCache.class), + mock(TaskSlotTable.class), + jobManagerTableMock, + jobLeaderService, + testingFatalErrorHandler); + try { + taskExecutor.start(); + + ArgumentCaptor jobLeaderListenerArgumentCaptor = ArgumentCaptor.forClass(JobLeaderListener.class); + + verify(jobLeaderService).start(anyString(), any(RpcService.class), any(HighAvailabilityServices.class), jobLeaderListenerArgumentCaptor.capture()); + + JobLeaderListener taskExecutorListener = jobLeaderListenerArgumentCaptor.getValue(); + + taskExecutorListener.jobManagerGainedLeadership(jobId, jobMasterGateway, jobLeaderId, registrationMessage); + + // duplicate job manager gained leadership message + taskExecutorListener.jobManagerGainedLeadership(jobId, jobMasterGateway, jobLeaderId, registrationMessage); + + ArgumentCaptor jobManagerConnectionArgumentCaptor = ArgumentCaptor.forClass(JobManagerConnection.class); + + verify(jobManagerTableMock, Mockito.timeout(verificationTimeout).times(1)).put(eq(jobId), jobManagerConnectionArgumentCaptor.capture()); + + JobManagerConnection jobManagerConnection = jobManagerConnectionArgumentCaptor.getValue(); + + assertEquals(jobMasterGateway, jobManagerConnection.getJobManagerGateway()); + + testingFatalErrorHandler.rethrowError(); + } finally { + taskExecutor.shutDown(); + taskExecutor.getTerminationFuture().get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } } } From 84c2a928cc4c5ee612bf57a1b944362d9114c92c Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 24 Aug 2017 19:16:10 +0200 Subject: [PATCH 087/129] [FLINK-7507] [dispatcher] Fence Dispatcher Let the Dispatcher extend the FencedRpcEndpoint and introduce DispatcherId which replaces the UUID as leader id/fencing token. This closes #4584. --- .../flink/runtime/dispatcher/Dispatcher.java | 44 +++++--------- .../runtime/dispatcher/DispatcherGateway.java | 7 +-- .../runtime/dispatcher/DispatcherId.java | 57 +++++++++++++++++++ .../runtime/dispatcher/DispatcherTest.java | 2 +- 4 files changed, 74 insertions(+), 36 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherId.java 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 e7e1ec2124cf1..00cbb2fc16140 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 @@ -28,7 +28,6 @@ import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.LeaderIdMismatchException; import org.apache.flink.runtime.highavailability.RunningJobsRegistry; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; @@ -40,7 +39,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.FencedRpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.util.ExceptionUtils; @@ -60,7 +59,7 @@ * the jobs and to recover them in case of a master failure. Furthermore, it knows * about the state of the Flink session cluster. */ -public abstract class Dispatcher extends RpcEndpoint implements DispatcherGateway, LeaderContender { +public abstract class Dispatcher extends FencedRpcEndpoint implements DispatcherGateway, LeaderContender { public static final String DISPATCHER_NAME = "dispatcher"; @@ -80,8 +79,6 @@ public abstract class Dispatcher extends RpcEndpoint implements DispatcherGatewa private final LeaderElectionService leaderElectionService; - private volatile UUID leaderSessionId; - protected Dispatcher( RpcService rpcService, String endpointId, @@ -91,7 +88,7 @@ protected Dispatcher( HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { - super(rpcService, endpointId); + super(rpcService, endpointId, DispatcherId.generate()); this.configuration = Preconditions.checkNotNull(configuration); this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices); @@ -106,9 +103,6 @@ protected Dispatcher( jobManagerRunners = new HashMap<>(16); leaderElectionService = highAvailabilityServices.getDispatcherLeaderElectionService(); - - // we are not the leader when this object is created - leaderSessionId = null; } //------------------------------------------------------ @@ -156,13 +150,7 @@ public void start() throws Exception { //------------------------------------------------------ @Override - public CompletableFuture submitJob(JobGraph jobGraph, UUID leaderSessionId, Time timeout) { - - try { - validateLeaderSessionId(leaderSessionId); - } catch (LeaderIdMismatchException e) { - return FutureUtils.completedExceptionally(e); - } + public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) { final JobID jobId = jobGraph.getJobID(); @@ -274,8 +262,6 @@ private void clearState() { private void recoverJobs() { log.info("Recovering all persisted jobs."); - final UUID currentLeaderSessionId = leaderSessionId; - getRpcService().execute( () -> { final Collection jobIds; @@ -291,7 +277,7 @@ private void recoverJobs() { try { SubmittedJobGraph submittedJobGraph = submittedJobGraphStore.recoverJobGraph(jobId); - runAsync(() -> submitJob(submittedJobGraph.getJobGraph(), currentLeaderSessionId, RpcUtils.INF_TIMEOUT)); + runAsync(() -> submitJob(submittedJobGraph.getJobGraph(), RpcUtils.INF_TIMEOUT)); } catch (Exception e) { log.error("Could not recover the job graph for " + jobId + '.', e); } @@ -304,12 +290,6 @@ private void onFatalError(Throwable throwable) { fatalErrorHandler.onFatalError(throwable); } - private void validateLeaderSessionId(UUID leaderSessionID) throws LeaderIdMismatchException { - if (this.leaderSessionId == null || !this.leaderSessionId.equals(leaderSessionID)) { - throw new LeaderIdMismatchException(this.leaderSessionId, leaderSessionID); - } - } - protected abstract JobManagerRunner createJobManagerRunner( ResourceID resourceId, JobGraph jobGraph, @@ -333,16 +313,18 @@ protected abstract JobManagerRunner createJobManagerRunner( */ @Override public void grantLeadership(final UUID newLeaderSessionID) { - runAsync( + runAsyncWithoutFencing( () -> { - log.info("Dispatcher {} was granted leadership with leader session ID {}", getAddress(), newLeaderSessionID); + final DispatcherId dispatcherId = new DispatcherId(newLeaderSessionID); + + log.info("Dispatcher {} was granted leadership with fencing token {}", getAddress(), dispatcherId); // clear the state if we've been the leader before - if (leaderSessionId != null) { + if (getFencingToken() != null) { clearState(); } - leaderSessionId = newLeaderSessionID; + setFencingToken(dispatcherId); // confirming the leader session ID might be blocking, getRpcService().execute( @@ -357,10 +339,12 @@ public void grantLeadership(final UUID newLeaderSessionID) { */ @Override public void revokeLeadership() { - runAsync( + runAsyncWithoutFencing( () -> { log.info("Dispatcher {} was revoked leadership.", getAddress()); clearState(); + + setFencingToken(DispatcherId.generate()); }); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java index 669f616a704db..09254c3ee4756 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherGateway.java @@ -22,29 +22,26 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import java.util.Collection; -import java.util.UUID; import java.util.concurrent.CompletableFuture; /** * Gateway for the Dispatcher component. */ -public interface DispatcherGateway extends RpcGateway { +public interface DispatcherGateway extends FencedRpcGateway { /** * Submit a job to the dispatcher. * * @param jobGraph JobGraph to submit - * @param leaderSessionId leader session id * @param timeout RPC timeout * @return A future acknowledge if the submission succeeded */ CompletableFuture submitJob( JobGraph jobGraph, - UUID leaderSessionId, @RpcTimeout Time timeout); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherId.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherId.java new file mode 100644 index 0000000000000..e5630904ce3ed --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherId.java @@ -0,0 +1,57 @@ +/* + * 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.dispatcher; + +import org.apache.flink.util.AbstractID; + +import java.util.UUID; + +/** + * Fencing token of the {@link Dispatcher}. + */ +public class DispatcherId extends AbstractID { + + private static final long serialVersionUID = -1654056277003743966L; + + public DispatcherId(byte[] bytes) { + super(bytes); + } + + public DispatcherId(long lowerPart, long upperPart) { + super(lowerPart, upperPart); + } + + public DispatcherId(AbstractID id) { + super(id); + } + + public DispatcherId() {} + + public DispatcherId(UUID uuid) { + this(uuid.getLeastSignificantBits(), uuid.getMostSignificantBits()); + } + + public UUID toUUID() { + return new UUID(getUpperPart(), getLowerPart()); + } + + public static DispatcherId generate() { + return new DispatcherId(); + } +} 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 091608c1e4bcf..884668624f00b 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 @@ -113,7 +113,7 @@ public void testJobSubmission() throws Exception { DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - CompletableFuture acknowledgeFuture = dispatcherGateway.submitJob(jobGraph, HighAvailabilityServices.DEFAULT_LEADER_ID, timeout); + CompletableFuture acknowledgeFuture = dispatcherGateway.submitJob(jobGraph, timeout); acknowledgeFuture.get(); From e70de0eb8f2f2a89cf74aa4bc6d1501e07e22d43 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 24 Aug 2017 15:11:00 +0200 Subject: [PATCH 088/129] [FLINK-7504] Fence the ResourceManager Properly fences the ResourceManager by letting it extend the FencedRpcEndpoint. Moreover, this PR introduces a ResourceManagerId which replaces the UUID as leader id/fencing token. This will give us more type safety when defining rpc interfaces. This closes #4582. --- .../MesosResourceManagerTest.java | 7 +- .../flink/runtime/instance/SlotPool.java | 10 +- .../runtime/instance/SlotPoolGateway.java | 4 +- .../flink/runtime/jobmaster/JobMaster.java | 49 +-- .../runtime/jobmaster/JobMasterGateway.java | 5 +- .../JobMasterRegistrationSuccess.java | 15 +- .../runtime/minicluster/MiniCluster.java | 5 +- .../registration/RetryingRegistration.java | 12 +- .../resourcemanager/ResourceManager.java | 314 ++++++------------ .../ResourceManagerGateway.java | 38 +-- .../resourcemanager/ResourceManagerId.java | 58 ++++ .../slotmanager/SlotManager.java | 22 +- .../taskexecutor/JobLeaderService.java | 7 +- .../runtime/taskexecutor/TaskExecutor.java | 35 +- .../taskexecutor/TaskExecutorGateway.java | 8 +- ...skExecutorToResourceManagerConnection.java | 20 +- .../clusterframework/ResourceManagerTest.java | 7 +- .../flink/runtime/instance/SlotPoolTest.java | 14 +- .../runtime/jobmaster/JobMasterTest.java | 9 +- .../runtime/metrics/MetricRegistryTest.java | 6 +- .../ResourceManagerHATest.java | 37 ++- .../ResourceManagerJobMasterTest.java | 68 ++-- .../ResourceManagerTaskExecutorTest.java | 40 ++- .../slotmanager/SlotManagerTest.java | 115 ++++--- .../slotmanager/SlotProtocolTest.java | 14 +- .../taskexecutor/TaskExecutorITCase.java | 3 +- .../taskexecutor/TaskExecutorTest.java | 48 ++- 27 files changed, 462 insertions(+), 508 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerId.java 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 cf0c913536475..02b043e7e8758 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 @@ -54,6 +54,7 @@ import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.slotmanager.ResourceManagerActions; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; @@ -326,7 +327,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { slotManagerStarted.complete(true); return null; } - }).when(slotManager).start(any(UUID.class), any(Executor.class), any(ResourceManagerActions.class)); + }).when(slotManager).start(any(ResourceManagerId.class), any(Executor.class), any(ResourceManagerActions.class)); when(slotManager.registerSlotRequest(any(SlotRequest.class))).thenReturn(true); } @@ -441,7 +442,7 @@ public void startResourceManager() throws Exception { */ public void registerJobMaster(MockJobMaster jobMaster) throws Exception { CompletableFuture registration = resourceManager.registerJobManager( - rmServices.rmLeaderSessionId, jobMaster.leaderSessionID, jobMaster.resourceID, jobMaster.address, jobMaster.jobID, timeout); + jobMaster.leaderSessionID, jobMaster.resourceID, jobMaster.address, jobMaster.jobID, timeout); assertTrue(registration.get() instanceof JobMasterRegistrationSuccess); } @@ -617,7 +618,7 @@ public void testWorkerStarted() throws Exception { // send registration message CompletableFuture successfulFuture = - resourceManager.registerTaskExecutor(rmServices.rmLeaderSessionId, task1Executor.address, task1Executor.resourceID, slotReport, timeout); + resourceManager.registerTaskExecutor(task1Executor.address, task1Executor.resourceID, slotReport, timeout); RegistrationResponse response = successfulFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(response instanceof TaskExecutorRegistrationSuccess); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java index bf3de25859c42..326e3a2e68d50 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java @@ -121,9 +121,6 @@ public class SlotPool extends RpcEndpoint implements SlotPoolGateway { /** the leader id of job manager */ private UUID jobManagerLeaderId; - /** The leader id of resource manager */ - private UUID resourceManagerLeaderId; - /** The gateway to communicate with resource manager */ private ResourceManagerGateway resourceManagerGateway; @@ -199,7 +196,6 @@ public void suspend() { // do not accept any requests jobManagerLeaderId = null; - resourceManagerLeaderId = null; resourceManagerGateway = null; // Clear (but not release!) the available slots. The TaskManagers should re-register them @@ -240,8 +236,7 @@ public SlotProvider getSlotProvider() { // ------------------------------------------------------------------------ @Override - public void connectToResourceManager(UUID resourceManagerLeaderId, ResourceManagerGateway resourceManagerGateway) { - this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId); + public void connectToResourceManager(ResourceManagerGateway resourceManagerGateway) { this.resourceManagerGateway = checkNotNull(resourceManagerGateway); // work on all slots waiting for this connection @@ -255,7 +250,6 @@ public void connectToResourceManager(UUID resourceManagerLeaderId, ResourceManag @Override public void disconnectResourceManager() { - this.resourceManagerLeaderId = null; this.resourceManagerGateway = null; } @@ -319,7 +313,7 @@ private void requestSlotFromResourceManager( pendingRequests.put(allocationID, new PendingRequest(allocationID, future, resources)); CompletableFuture rmResponse = resourceManagerGateway.requestSlot( - jobManagerLeaderId, resourceManagerLeaderId, + jobManagerLeaderId, new SlotRequest(jobId, allocationID, resources, jobManagerAddress), resourceManagerRequestsTimeout); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java index 32a9af5c9ef06..06c4b120cca3d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPoolGateway.java @@ -33,7 +33,6 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import java.util.Collection; -import java.util.UUID; import java.util.concurrent.CompletableFuture; /** @@ -55,10 +54,9 @@ public interface SlotPoolGateway extends RpcGateway { * Connects the SlotPool to the given ResourceManager. After this method is called, the * SlotPool will be able to request resources from the given ResourceManager. * - * @param resourceManagerLeaderId The leader session ID of the resource manager. * @param resourceManagerGateway The RPC gateway for the resource manager. */ - void connectToResourceManager(UUID resourceManagerLeaderId, ResourceManagerGateway resourceManagerGateway); + void connectToResourceManager(ResourceManagerGateway resourceManagerGateway); /** * Disconnects the slot pool from its current Resource Manager. After this call, the pool will not 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 55bc8f8677b92..a05242aafb42f 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 @@ -81,6 +81,7 @@ import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.registration.RetryingRegistration; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; @@ -756,17 +757,17 @@ public void requestHeartbeat(ResourceID resourceID, Void payload) { @Override public void disconnectResourceManager( final UUID jobManagerLeaderId, - final UUID resourceManagerLeaderId, + final ResourceManagerId resourceManagerId, final Exception cause) { try { validateLeaderSessionId(jobManagerLeaderId); } catch (LeaderIdMismatchException e) { - log.warn("Cannot disconnect resource manager " + resourceManagerLeaderId + '.', e); + log.warn("Cannot disconnect resource manager " + resourceManagerId + '.', e); } if (resourceManagerConnection != null - && resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderId)) { + && resourceManagerConnection.getTargetLeaderId().equals(resourceManagerId)) { closeResourceManagerConnection(cause); } } @@ -944,11 +945,11 @@ private void jobStatusChanged(final JobStatus newJobStatus, long timestamp, fina } } - private void notifyOfNewResourceManagerLeader(final String resourceManagerAddress, final UUID resourceManagerLeaderId) { + private void notifyOfNewResourceManagerLeader(final String resourceManagerAddress, final ResourceManagerId resourceManagerId) { if (resourceManagerConnection != null) { if (resourceManagerAddress != null) { - if (resourceManagerAddress.equals(resourceManagerConnection.getTargetAddress()) - && resourceManagerLeaderId.equals(resourceManagerConnection.getTargetLeaderId())) { + if (Objects.equals(resourceManagerAddress, resourceManagerConnection.getTargetAddress()) + && Objects.equals(resourceManagerId, resourceManagerConnection.getTargetLeaderId())) { // both address and leader id are not changed, we can keep the old connection return; } @@ -974,7 +975,7 @@ private void notifyOfNewResourceManagerLeader(final String resourceManagerAddres getAddress(), leaderSessionID, resourceManagerAddress, - resourceManagerLeaderId, + resourceManagerId, executor); resourceManagerConnection.start(); @@ -982,17 +983,17 @@ private void notifyOfNewResourceManagerLeader(final String resourceManagerAddres } private void establishResourceManagerConnection(final JobMasterRegistrationSuccess success) { - final UUID resourceManagerLeaderId = success.getResourceManagerLeaderId(); + final ResourceManagerId resourceManagerId = success.getResourceManagerId(); // verify the response with current connection if (resourceManagerConnection != null - && resourceManagerConnection.getTargetLeaderId().equals(resourceManagerLeaderId)) { + && Objects.equals(resourceManagerConnection.getTargetLeaderId(), resourceManagerId)) { - log.info("JobManager successfully registered at ResourceManager, leader id: {}.", resourceManagerLeaderId); + log.info("JobManager successfully registered at ResourceManager, leader id: {}.", resourceManagerId); final ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway(); - slotPoolGateway.connectToResourceManager(resourceManagerLeaderId, resourceManagerGateway); + slotPoolGateway.connectToResourceManager(resourceManagerGateway); resourceManagerHeartbeatManager.monitorTarget(success.getResourceManagerResourceId(), new HeartbeatTarget() { @Override @@ -1005,6 +1006,9 @@ public void requestHeartbeat(ResourceID resourceID, Void payload) { // request heartbeat will never be called on the job manager side } }); + } else { + log.debug("Ignoring resource manager connection to {} because its a duplicate or outdated.", resourceManagerId); + } } @@ -1038,12 +1042,10 @@ private class ResourceManagerLeaderListener implements LeaderRetrievalListener { @Override public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { - runAsync(new Runnable() { - @Override - public void run() { - notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID); - } - }); + runAsync( + () -> notifyOfNewResourceManagerLeader( + leaderAddress, + leaderSessionID != null ? new ResourceManagerId(leaderSessionID) : null)); } @Override @@ -1055,7 +1057,7 @@ public void handleError(final Exception exception) { //---------------------------------------------------------------------------------------------- private class ResourceManagerConnection - extends RegisteredRpcConnection + extends RegisteredRpcConnection { private final JobID jobID; @@ -1074,10 +1076,10 @@ private class ResourceManagerConnection final String jobManagerRpcAddress, final UUID jobManagerLeaderID, final String resourceManagerAddress, - final UUID resourceManagerLeaderID, + final ResourceManagerId resourceManagerId, final Executor executor) { - super(log, resourceManagerAddress, resourceManagerLeaderID, executor); + super(log, resourceManagerAddress, resourceManagerId, executor); this.jobID = checkNotNull(jobID); this.jobManagerResourceID = checkNotNull(jobManagerResourceID); this.jobManagerRpcAddress = checkNotNull(jobManagerRpcAddress); @@ -1085,19 +1087,18 @@ private class ResourceManagerConnection } @Override - protected RetryingRegistration generateRegistration() { - return new RetryingRegistration( + protected RetryingRegistration generateRegistration() { + return new RetryingRegistration( log, getRpcService(), "ResourceManager", ResourceManagerGateway.class, getTargetAddress(), getTargetLeaderId()) { @Override protected CompletableFuture invokeRegistration( - ResourceManagerGateway gateway, UUID leaderId, long timeoutMillis) throws Exception + ResourceManagerGateway gateway, ResourceManagerId fencingToken, long timeoutMillis) throws Exception { Time timeout = Time.milliseconds(timeoutMillis); return gateway.registerJobManager( - leaderId, jobManagerLeaderID, jobManagerResourceID, jobManagerRpcAddress, 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 bfa29308c2ae6..b39f419bbd446 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 @@ -36,6 +36,7 @@ import org.apache.flink.runtime.query.KvStateLocation; import org.apache.flink.runtime.query.KvStateServerAddress; import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.state.internal.InternalKvState; import org.apache.flink.runtime.state.KeyGroupRange; @@ -123,12 +124,12 @@ CompletableFuture scheduleOrUpdateConsumers( * Disconnects the resource manager from the job manager because of the given cause. * * @param jobManagerLeaderId identifying the job manager leader id - * @param resourceManagerLeaderId identifying the resource manager leader id + * @param resourceManagerId identifying the resource manager leader id * @param cause of the disconnect */ void disconnectResourceManager( final UUID jobManagerLeaderId, - final UUID resourceManagerLeaderId, + final ResourceManagerId resourceManagerId, final Exception cause); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java index a7a622465eccb..94ecfd2e92574 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterRegistrationSuccess.java @@ -20,8 +20,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.registration.RegistrationResponse; - -import java.util.UUID; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -34,16 +33,16 @@ public class JobMasterRegistrationSuccess extends RegistrationResponse.Success { private final long heartbeatInterval; - private final UUID resourceManagerLeaderId; + private final ResourceManagerId resourceManagerId; private final ResourceID resourceManagerResourceId; public JobMasterRegistrationSuccess( final long heartbeatInterval, - final UUID resourceManagerLeaderId, + final ResourceManagerId resourceManagerId, final ResourceID resourceManagerResourceId) { this.heartbeatInterval = heartbeatInterval; - this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId); + this.resourceManagerId = checkNotNull(resourceManagerId); this.resourceManagerResourceId = checkNotNull(resourceManagerResourceId); } @@ -56,8 +55,8 @@ public long getHeartbeatInterval() { return heartbeatInterval; } - public UUID getResourceManagerLeaderId() { - return resourceManagerLeaderId; + public ResourceManagerId getResourceManagerId() { + return resourceManagerId; } public ResourceID getResourceManagerResourceId() { @@ -68,7 +67,7 @@ public ResourceID getResourceManagerResourceId() { public String toString() { return "JobMasterRegistrationSuccess{" + "heartbeatInterval=" + heartbeatInterval + - ", resourceManagerLeaderId=" + resourceManagerLeaderId + + ", resourceManagerLeaderId=" + resourceManagerId + ", resourceManagerResourceId=" + resourceManagerResourceId + '}'; } 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 9a4a76ab80af5..95f430c0ee245 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 @@ -38,6 +38,7 @@ import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.MetricRegistryConfiguration; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.ResourceManagerRunner; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -419,14 +420,14 @@ public void waitUntilTaskManagerRegistrationsComplete() throws Exception { final LeaderAddressAndId addressAndId = addressAndIdFuture.get(); final ResourceManagerGateway resourceManager = - commonRpcService.connect(addressAndId.leaderAddress(), ResourceManagerGateway.class).get(); + commonRpcService.connect(addressAndId.leaderAddress(), new ResourceManagerId(addressAndId.leaderId()), ResourceManagerGateway.class).get(); final int numTaskManagersToWaitFor = taskManagers.length; // poll and wait until enough TaskManagers are available while (true) { int numTaskManagersAvailable = - resourceManager.getNumberOfRegisteredTaskManagers(addressAndId.leaderId()).get(); + resourceManager.getNumberOfRegisteredTaskManagers().get(); if (numTaskManagersAvailable >= numTaskManagersToWaitFor) { break; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java index be30c6838dbe5..ce4a798b0d35c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.registration; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; @@ -176,7 +177,16 @@ protected abstract CompletableFuture invokeRegistration( public void startRegistration() { try { // trigger resolution of the resource manager address to a callable gateway - CompletableFuture resourceManagerFuture = rpcService.connect(targetAddress, targetType); + final CompletableFuture resourceManagerFuture; + + if (FencedRpcGateway.class.isAssignableFrom(targetType)) { + resourceManagerFuture = (CompletableFuture) rpcService.connect( + targetAddress, + fencingToken, + targetType.asSubclass(FencedRpcGateway.class)); + } else { + resourceManagerFuture = rpcService.connect(targetAddress, targetType); + } // upon success, start the registration attempts CompletableFuture resourceManagerAcceptFuture = resourceManagerFuture.thenAcceptAsync( 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 6b2c8980cf836..659b3d4073d33 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 @@ -33,7 +33,6 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.HeartbeatTarget; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.LeaderIdMismatchException; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; @@ -50,11 +49,10 @@ import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerException; import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.FencedRpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; import org.apache.flink.runtime.taskexecutor.SlotReport; -import org.apache.flink.runtime.taskexecutor.TaskExecutor; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.util.ExceptionUtils; @@ -77,12 +75,12 @@ * *

    It offers the following methods as part of its rpc interface to interact with him remotely: *

      - *
    • {@link #registerJobManager(UUID, UUID, ResourceID, String, JobID, Time)} registers a {@link JobMaster} at the resource manager
    • - *
    • {@link #requestSlot(UUID, UUID, SlotRequest, Time)} requests a slot from the resource manager
    • + *
    • {@link #registerJobManager(UUID, ResourceID, String, JobID, Time)} registers a {@link JobMaster} at the resource manager
    • + *
    • {@link #requestSlot(UUID, SlotRequest, Time)} requests a slot from the resource manager
    • *
    */ public abstract class ResourceManager - extends RpcEndpoint + extends FencedRpcEndpoint implements ResourceManagerGateway, LeaderContender { public static final String RESOURCE_MANAGER_NAME = "resourcemanager"; @@ -126,9 +124,6 @@ public abstract class ResourceManager /** The service to elect a ResourceManager leader. */ private LeaderElectionService leaderElectionService; - /** ResourceManager's leader session id which is updated on leader election. */ - private volatile UUID leaderSessionId; - /** All registered listeners for status updates of the ResourceManager. */ private ConcurrentMap infoMessageListeners; @@ -144,7 +139,7 @@ public ResourceManager( JobLeaderIdService jobLeaderIdService, FatalErrorHandler fatalErrorHandler) { - super(rpcService, resourceManagerEndpointId); + super(rpcService, resourceManagerEndpointId, ResourceManagerId.generate()); this.resourceId = checkNotNull(resourceId); this.resourceManagerConfiguration = checkNotNull(resourceManagerConfiguration); @@ -169,7 +164,6 @@ public ResourceManager( this.jobManagerRegistrations = new HashMap<>(4); this.jmResourceIdRegistrations = new HashMap<>(4); this.taskExecutors = new HashMap<>(8); - this.leaderSessionId = null; infoMessageListeners = new ConcurrentHashMap<>(8); } @@ -246,147 +240,109 @@ public void postStop() throws Exception { @Override public CompletableFuture registerJobManager( - final UUID resourceManagerLeaderId, final UUID jobManagerLeaderId, final ResourceID jobManagerResourceId, final String jobManagerAddress, final JobID jobId, final Time timeout) { - checkNotNull(resourceManagerLeaderId); checkNotNull(jobManagerLeaderId); checkNotNull(jobManagerResourceId); checkNotNull(jobManagerAddress); checkNotNull(jobId); - if (isValid(resourceManagerLeaderId)) { - if (!jobLeaderIdService.containsJob(jobId)) { - try { - jobLeaderIdService.addJob(jobId); - } catch (Exception e) { - ResourceManagerException exception = new ResourceManagerException("Could not add the job " + - jobId + " to the job id leader service.", e); + if (!jobLeaderIdService.containsJob(jobId)) { + try { + jobLeaderIdService.addJob(jobId); + } catch (Exception e) { + ResourceManagerException exception = new ResourceManagerException("Could not add the job " + + jobId + " to the job id leader service.", e); onFatalError(exception); - log.error("Could not add job {} to job leader id service.", jobId, e); - return FutureUtils.completedExceptionally(exception); - } + log.error("Could not add job {} to job leader id service.", jobId, e); + return FutureUtils.completedExceptionally(exception); } + } - log.info("Registering job manager {}@{} for job {}.", jobManagerLeaderId, jobManagerAddress, jobId); + log.info("Registering job manager {}@{} for job {}.", jobManagerLeaderId, jobManagerAddress, jobId); - CompletableFuture jobLeaderIdFuture; + CompletableFuture jobLeaderIdFuture; - try { - jobLeaderIdFuture = jobLeaderIdService.getLeaderId(jobId); - } catch (Exception e) { - // we cannot check the job leader id so let's fail - // TODO: Maybe it's also ok to skip this check in case that we cannot check the leader id - ResourceManagerException exception = new ResourceManagerException("Cannot obtain the " + - "job leader id future to verify the correct job leader.", e); + try { + jobLeaderIdFuture = jobLeaderIdService.getLeaderId(jobId); + } catch (Exception e) { + // we cannot check the job leader id so let's fail + // TODO: Maybe it's also ok to skip this check in case that we cannot check the leader id + ResourceManagerException exception = new ResourceManagerException("Cannot obtain the " + + "job leader id future to verify the correct job leader.", e); onFatalError(exception); - log.debug("Could not obtain the job leader id future to verify the correct job leader."); - return FutureUtils.completedExceptionally(exception); - } - - CompletableFuture jobMasterGatewayFuture = getRpcService().connect(jobManagerAddress, JobMasterGateway.class); - - CompletableFuture registrationResponseFuture = jobMasterGatewayFuture.thenCombineAsync( - jobLeaderIdFuture, - (JobMasterGateway jobMasterGateway, UUID jobLeaderId) -> { - if (isValid(resourceManagerLeaderId)) { - if (Objects.equals(jobLeaderId, jobManagerLeaderId)) { - return registerJobMasterInternal( - jobMasterGateway, - jobLeaderId, - jobId, - jobManagerAddress, - jobManagerResourceId); - } else { - log.debug("The job manager leader id {} did not match the job " + - "leader id {}.", jobManagerLeaderId, jobLeaderId); - return new RegistrationResponse.Decline("Job manager leader id did not match."); - } - } else { - log.debug("The resource manager leader id changed {}. Discarding job " + - "manager registration from {}.", getLeaderSessionId(), jobManagerAddress); - return new RegistrationResponse.Decline("Resource manager leader id changed."); - } - }, - getMainThreadExecutor()); + log.debug("Could not obtain the job leader id future to verify the correct job leader."); + return FutureUtils.completedExceptionally(exception); + } - // handle exceptions which might have occurred in one of the futures inputs of combine - return registrationResponseFuture.handleAsync( - (RegistrationResponse registrationResponse, Throwable throwable) -> { - if (throwable != null) { - if (log.isDebugEnabled()) { - log.debug("Registration of job manager {}@{} failed.", jobManagerLeaderId, jobManagerAddress, throwable); - } else { - log.info("Registration of job manager {}@{} failed.", jobManagerLeaderId, jobManagerAddress); - } + CompletableFuture jobMasterGatewayFuture = getRpcService().connect(jobManagerAddress, JobMasterGateway.class); - return new RegistrationResponse.Decline(throwable.getMessage()); + CompletableFuture registrationResponseFuture = jobMasterGatewayFuture.thenCombineAsync( + jobLeaderIdFuture, + (JobMasterGateway jobMasterGateway, UUID jobLeaderId) -> { + if (Objects.equals(jobLeaderId, jobManagerLeaderId)) { + return registerJobMasterInternal( + jobMasterGateway, + jobLeaderId, + jobId, + jobManagerAddress, + jobManagerResourceId); + } else { + log.debug("The job manager leader id {} did not match the job " + + "leader id {}.", jobManagerLeaderId, jobLeaderId); + return new RegistrationResponse.Decline("Job manager leader id did not match."); + } + }, + getMainThreadExecutor()); + + // handle exceptions which might have occurred in one of the futures inputs of combine + return registrationResponseFuture.handleAsync( + (RegistrationResponse registrationResponse, Throwable throwable) -> { + if (throwable != null) { + if (log.isDebugEnabled()) { + log.debug("Registration of job manager {}@{} failed.", jobManagerLeaderId, jobManagerAddress, throwable); } else { - return registrationResponse; + log.info("Registration of job manager {}@{} failed.", jobManagerLeaderId, jobManagerAddress); } - }, - getRpcService().getExecutor()); - } else { - log.debug("Discard register job manager message from {}, because the leader id " + - "{} did not match the expected leader id {}.", jobManagerAddress, - resourceManagerLeaderId, leaderSessionId); - return CompletableFuture.completedFuture( - new RegistrationResponse.Decline("Resource manager leader id did not match.")); - } + return new RegistrationResponse.Decline(throwable.getMessage()); + } else { + return registrationResponse; + } + }, + getRpcService().getExecutor()); } - /** - * Register a {@link TaskExecutor} at the resource manager. - * - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param taskExecutorAddress The address of the TaskExecutor that registers - * @param taskExecutorResourceId The resource ID of the TaskExecutor that registers - * - * @return The response by the ResourceManager. - */ @Override public CompletableFuture registerTaskExecutor( - final UUID resourceManagerLeaderId, final String taskExecutorAddress, final ResourceID taskExecutorResourceId, final SlotReport slotReport, final Time timeout) { - if (Objects.equals(leaderSessionId, resourceManagerLeaderId)) { - CompletableFuture taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class); + CompletableFuture taskExecutorGatewayFuture = getRpcService().connect(taskExecutorAddress, TaskExecutorGateway.class); - return taskExecutorGatewayFuture.handleAsync( - (TaskExecutorGateway taskExecutorGateway, Throwable throwable) -> { - if (throwable != null) { - return new RegistrationResponse.Decline(throwable.getMessage()); - } else { - return registerTaskExecutorInternal( - taskExecutorGateway, - taskExecutorAddress, - taskExecutorResourceId, - slotReport); - } - }, - getMainThreadExecutor()); - } else { - log.warn("Discard registration from TaskExecutor {} at ({}) because the expected leader session ID {} did " + - "not equal the received leader session ID {}", - taskExecutorResourceId, taskExecutorAddress, leaderSessionId, resourceManagerLeaderId); - - return CompletableFuture.completedFuture( - new RegistrationResponse.Decline("Discard registration because the leader id " + - resourceManagerLeaderId + " does not match the expected leader id " + - leaderSessionId + '.')); - } + return taskExecutorGatewayFuture.handleAsync( + (TaskExecutorGateway taskExecutorGateway, Throwable throwable) -> { + if (throwable != null) { + return new RegistrationResponse.Decline(throwable.getMessage()); + } else { + return registerTaskExecutorInternal( + taskExecutorGateway, + taskExecutorAddress, + taskExecutorResourceId, + slotReport); + } + }, + getMainThreadExecutor()); } @Override @@ -409,23 +365,12 @@ public void disconnectJobManager(final JobID jobId, final Exception cause) { closeJobManagerConnection(jobId, cause); } - /** - * Requests a slot from the resource manager. - * - * @param slotRequest Slot request - * @return Slot assignment - */ @Override public CompletableFuture requestSlot( UUID jobMasterLeaderID, - UUID resourceManagerLeaderID, SlotRequest slotRequest, final Time timeout) { - if (!Objects.equals(resourceManagerLeaderID, leaderSessionId)) { - return FutureUtils.completedExceptionally(new LeaderSessionIDException(resourceManagerLeaderID, leaderSessionId)); - } - JobID jobId = slotRequest.getJobId(); JobManagerRegistration jobManagerRegistration = jobManagerRegistrations.get(jobId); @@ -452,41 +397,27 @@ public CompletableFuture requestSlot( } } - /** - * Notification from a TaskExecutor that a slot has become available. - * - * @param resourceManagerLeaderId TaskExecutor's resource manager leader id - * @param instanceID TaskExecutor's instance id - * @param slotId The slot id of the available slot - */ @Override public void notifySlotAvailable( - final UUID resourceManagerLeaderId, final InstanceID instanceID, final SlotID slotId, final AllocationID allocationId) { - if (Objects.equals(resourceManagerLeaderId, leaderSessionId)) { - final ResourceID resourceId = slotId.getResourceID(); - WorkerRegistration registration = taskExecutors.get(resourceId); + final ResourceID resourceId = slotId.getResourceID(); + WorkerRegistration registration = taskExecutors.get(resourceId); - if (registration != null) { - InstanceID registrationId = registration.getInstanceID(); + if (registration != null) { + InstanceID registrationId = registration.getInstanceID(); - if (Objects.equals(registrationId, instanceID)) { - slotManager.freeSlot(slotId, allocationId); - } else { - log.debug("Invalid registration id for slot available message. This indicates an" + - " outdated request."); - } + if (Objects.equals(registrationId, instanceID)) { + slotManager.freeSlot(slotId, allocationId); } else { - log.debug("Could not find registration for resource id {}. Discarding the slot available" + - "message {}.", resourceId, slotId); + log.debug("Invalid registration id for slot available message. This indicates an" + + " outdated request."); } } else { - log.debug("Discarding notify slot available message for slot {}, because the " + - "leader id {} did not match the expected leader id {}.", slotId, - resourceManagerLeaderId, leaderSessionId); + log.debug("Could not find registration for resource id {}. Discarding the slot available" + + "message {}.", resourceId, slotId); } } @@ -545,27 +476,8 @@ public void shutDownCluster(final ApplicationStatus finalStatus, final String op } @Override - public CompletableFuture getNumberOfRegisteredTaskManagers(UUID requestLeaderSessionId) { - if (Objects.equals(leaderSessionId, requestLeaderSessionId)) { - return CompletableFuture.completedFuture(taskExecutors.size()); - } - else { - return FutureUtils.completedExceptionally(new LeaderIdMismatchException(leaderSessionId, requestLeaderSessionId)); - } - } - - // ------------------------------------------------------------------------ - // Testing methods - // ------------------------------------------------------------------------ - - /** - * Gets the leader session id of current resourceManager. - * - * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership. - */ - @VisibleForTesting - UUID getLeaderSessionId() { - return leaderSessionId; + public CompletableFuture getNumberOfRegisteredTaskManagers() { + return CompletableFuture.completedFuture(taskExecutors.size()); } // ------------------------------------------------------------------------ @@ -635,7 +547,7 @@ public void requestHeartbeat(ResourceID resourceID, Void payload) { return new JobMasterRegistrationSuccess( resourceManagerConfiguration.getHeartbeatInterval().toMilliseconds(), - getLeaderSessionId(), + getFencingToken(), resourceId); } @@ -706,8 +618,6 @@ private void clearState() { } catch (Exception e) { onFatalError(new ResourceManagerException("Could not properly clear the job leader id service.", e)); } - - leaderSessionId = null; } /** @@ -735,7 +645,7 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) { jmResourceIdRegistrations.remove(jobManagerResourceId); // tell the job manager about the disconnect - jobMasterGateway.disconnectResourceManager(jobManagerLeaderId, getLeaderSessionId(), cause); + jobMasterGateway.disconnectResourceManager(jobManagerLeaderId, getFencingToken(), cause); } else { log.debug("There was no registered job manager for job {}.", jobId); } @@ -765,17 +675,6 @@ protected void closeTaskManagerConnection(final ResourceID resourceID, final Exc } } - /** - * Checks whether the given resource manager leader id is matching the current leader id and - * not null. - * - * @param resourceManagerLeaderId to check - * @return True if the given leader id matches the actual leader id and is not null; otherwise false - */ - protected boolean isValid(UUID resourceManagerLeaderId) { - return Objects.equals(resourceManagerLeaderId, leaderSessionId); - } - protected void removeJob(JobID jobId) { try { jobLeaderIdService.removeJob(jobId); @@ -848,29 +747,26 @@ protected void onFatalError(Throwable t) { */ @Override public void grantLeadership(final UUID newLeaderSessionID) { - runAsync(new Runnable() { - @Override - public void run() { - log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), newLeaderSessionID); + runAsyncWithoutFencing( + () -> { + final ResourceManagerId newResourceManagerId = new ResourceManagerId(newLeaderSessionID); + + log.info("ResourceManager {} was granted leadership with fencing token {}", getAddress(), newResourceManagerId); // clear the state if we've been the leader before - if (leaderSessionId != null) { + if (getFencingToken() != null) { clearState(); } - leaderSessionId = newLeaderSessionID; + setFencingToken(newResourceManagerId); - slotManager.start(leaderSessionId, getMainThreadExecutor(), new ResourceManagerActionsImpl()); + slotManager.start(getFencingToken(), getMainThreadExecutor(), new ResourceManagerActionsImpl()); - getRpcService().execute(new Runnable() { - @Override - public void run() { + getRpcService().execute( + () -> // confirming the leader session ID might be blocking, - leaderElectionService.confirmLeaderSessionID(newLeaderSessionID); - } - }); - } - }); + leaderElectionService.confirmLeaderSessionID(newLeaderSessionID)); + }); } /** @@ -878,16 +774,18 @@ public void run() { */ @Override public void revokeLeadership() { - runAsync(new Runnable() { - @Override - public void run() { - log.info("ResourceManager {} was revoked leadership.", getAddress()); + runAsyncWithoutFencing( + () -> { + final ResourceManagerId newResourceManagerId = ResourceManagerId.generate(); + + log.info("ResourceManager {} was revoked leadership. Setting fencing token to {}.", getAddress(), newResourceManagerId); clearState(); + setFencingToken(newResourceManagerId); + slotManager.suspend(); - } - }); + }); } /** 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 1ba68932b45a5..ac81048acd781 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 @@ -26,11 +26,12 @@ import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.rpc.RpcGateway; +import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.taskexecutor.SlotReport; +import org.apache.flink.runtime.taskexecutor.TaskExecutor; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -38,69 +39,61 @@ /** * The {@link ResourceManager}'s RPC gateway interface. */ -public interface ResourceManagerGateway extends RpcGateway { +public interface ResourceManagerGateway extends FencedRpcGateway { /** * Register a {@link JobMaster} at the resource manager. * - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader * @param jobMasterLeaderId The fencing token for the JobMaster leader - * @param jobMasterResourceId The resource ID of the JobMaster that registers - * @param jobMasterAddress The address of the JobMaster that registers - * @param jobID The Job ID of the JobMaster that registers - * @param timeout Timeout for the future to complete + * @param jobMasterResourceId The resource ID of the JobMaster that registers + * @param jobMasterAddress The address of the JobMaster that registers + * @param jobId The Job ID of the JobMaster that registers + * @param timeout Timeout for the future to complete * @return Future registration response */ CompletableFuture registerJobManager( - UUID resourceManagerLeaderId, UUID jobMasterLeaderId, ResourceID jobMasterResourceId, String jobMasterAddress, - JobID jobID, + JobID jobId, @RpcTimeout Time timeout); /** * Requests a slot from the resource manager. * - * @param resourceManagerLeaderID leader if of the ResourceMaster * @param jobMasterLeaderID leader if of the JobMaster * @param slotRequest The slot to request * @return The confirmation that the slot gets allocated */ CompletableFuture requestSlot( - UUID resourceManagerLeaderID, UUID jobMasterLeaderID, SlotRequest slotRequest, @RpcTimeout Time timeout); /** - * Register a {@link org.apache.flink.runtime.taskexecutor.TaskExecutor} at the resource manager. + * Register a {@link TaskExecutor} at the resource manager. * - * @param resourceManagerLeaderId The fencing token for the ResourceManager leader - * @param taskExecutorAddress The address of the TaskExecutor that registers - * @param resourceID The resource ID of the TaskExecutor that registers - * @param slotReport The slot report containing free and allocated task slots - * @param timeout The timeout for the response. + * @param taskExecutorAddress The address of the TaskExecutor that registers + * @param resourceId The resource ID of the TaskExecutor that registers + * @param slotReport The slot report containing free and allocated task slots + * @param timeout The timeout for the response. * * @return The future to the response by the ResourceManager. */ CompletableFuture registerTaskExecutor( - UUID resourceManagerLeaderId, String taskExecutorAddress, - ResourceID resourceID, + ResourceID resourceId, SlotReport slotReport, @RpcTimeout Time timeout); /** * Sent by the TaskExecutor to notify the ResourceManager that a slot has become available. * - * @param resourceManagerLeaderId The ResourceManager leader id * @param instanceId TaskExecutor's instance id * @param slotID The SlotID of the freed slot * @param oldAllocationId to which the slot has been allocated */ void notifySlotAvailable( - UUID resourceManagerLeaderId, InstanceID instanceId, SlotID slotID, AllocationID oldAllocationId); @@ -130,10 +123,9 @@ void notifySlotAvailable( /** * Gets the currently registered number of TaskManagers. * - * @param leaderSessionId The leader session ID with which to address the ResourceManager. * @return The future to the number of registered TaskManagers. */ - CompletableFuture getNumberOfRegisteredTaskManagers(UUID leaderSessionId); + CompletableFuture getNumberOfRegisteredTaskManagers(); /** * Sends the heartbeat to resource manager from task manager diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerId.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerId.java new file mode 100644 index 0000000000000..3594e88e12249 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerId.java @@ -0,0 +1,58 @@ +/* + * 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.resourcemanager; + +import org.apache.flink.util.AbstractID; + +import java.util.UUID; + +/** + * Fencing token for the {@link ResourceManager}. + */ +public class ResourceManagerId extends AbstractID { + + private static final long serialVersionUID = -6042820142662137374L; + + public ResourceManagerId(byte[] bytes) { + super(bytes); + } + + public ResourceManagerId(long lowerPart, long upperPart) { + super(lowerPart, upperPart); + } + + public ResourceManagerId(AbstractID id) { + super(id); + } + + public ResourceManagerId() { + } + + public ResourceManagerId(UUID uuid) { + this(uuid.getLeastSignificantBits(), uuid.getMostSignificantBits()); + } + + public UUID toUUID() { + return new UUID(getUpperPart(), getLowerPart()); + } + + public static ResourceManagerId generate() { + return new ResourceManagerId(); + } +} 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 52182860be16a..d8eb47c5ce50b 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 @@ -27,6 +27,7 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection; @@ -35,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.AbstractID; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +47,6 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.Objects; -import java.util.UUID; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -64,7 +65,7 @@ * slots are currently not used) and pending slot requests time out triggering their release and * failure, respectively. */ -public class SlotManager implements AutoCloseable { +public class SlotManager implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class); /** Scheduled executor for timeouts */ @@ -94,8 +95,8 @@ public class SlotManager implements AutoCloseable { /** Map of pending/unfulfilled slot allocation requests */ private final HashMap pendingSlotRequests; - /** Leader id of the containing component */ - private UUID leaderId; + /** ResourceManager's id */ + private ResourceManagerId resourceManagerId; /** Executor for future callbacks which have to be "synchronized" */ private Executor mainThreadExecutor; @@ -126,7 +127,7 @@ public SlotManager( fulfilledSlotRequests = new HashMap<>(16); pendingSlotRequests = new HashMap<>(16); - leaderId = null; + resourceManagerId = null; resourceManagerActions = null; mainThreadExecutor = null; taskManagerTimeoutCheck = null; @@ -142,13 +143,14 @@ public SlotManager( /** * Starts the slot manager with the given leader id and resource manager actions. * - * @param newLeaderId to use for communication with the task managers + * @param newResourceManagerId to use for communication with the task managers + * @param newMainThreadExecutor to use to run code in the ResourceManager's main thread * @param newResourceManagerActions to use for resource (de-)allocations */ - public void start(UUID newLeaderId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) { + public void start(ResourceManagerId newResourceManagerId, Executor newMainThreadExecutor, ResourceManagerActions newResourceManagerActions) { LOG.info("Starting the SlotManager."); - leaderId = Preconditions.checkNotNull(newLeaderId); + this.resourceManagerId = Preconditions.checkNotNull(newResourceManagerId); mainThreadExecutor = Preconditions.checkNotNull(newMainThreadExecutor); resourceManagerActions = Preconditions.checkNotNull(newResourceManagerActions); @@ -204,7 +206,7 @@ public void suspend() { unregisterTaskManager(registeredTaskManager); } - leaderId = null; + resourceManagerId = null; resourceManagerActions = null; started = false; } @@ -643,7 +645,7 @@ private void allocateSlot(TaskManagerSlot taskManagerSlot, PendingSlotRequest pe pendingSlotRequest.getJobId(), allocationId, pendingSlotRequest.getTargetAddress(), - leaderId, + resourceManagerId, taskManagerRequestTimeout); requestFuture.whenComplete( 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 6d1f22c20acc9..f564df4db1bda 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 @@ -37,6 +37,7 @@ import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -325,7 +326,7 @@ protected RetryingRegistration @Override protected void onRegistrationSuccess(JMTMRegistrationSuccess success) { // filter out old registration attempts - if (getTargetLeaderId().equals(currentLeaderId)) { + if (Objects.equals(getTargetLeaderId(), currentLeaderId)) { log.info("Successful registration at job manager {} for job {}.", getTargetAddress(), jobId); jobLeaderListener.jobManagerGainedLeadership(jobId, getTargetGateway(), getTargetLeaderId(), success); @@ -337,8 +338,8 @@ protected void onRegistrationSuccess(JMTMRegistrationSuccess success) { @Override protected void onRegistrationFailure(Throwable failure) { // filter out old registration attempts - if (getTargetLeaderId().equals(currentLeaderId)) { - log.info("Failed to register at job manager {} for job {}.", getTargetAddress(), jobId); + if (Objects.equals(getTargetLeaderId(), currentLeaderId)) { + log.info("Failed to register at job manager {} for job {}.", getTargetAddress(), jobId); jobLeaderListener.handleError(failure); } else { log.debug("Obsolete JobManager registration failure from {} with leader session ID {}.", getTargetAddress(), getTargetLeaderId(), failure); 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 ef47ad4c6a49b..3b1a1b49b72b6 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 @@ -59,6 +59,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; @@ -585,29 +586,18 @@ public CompletableFuture confirmCheckpoint( // Slot allocation RPCs // ---------------------------------------------------------------------- - /** - * Requests a slot from the TaskManager - * - * @param slotId identifying the requested slot - * @param jobId identifying the job for which the request is issued - * @param allocationId id for the request - * @param targetAddress of the job manager requesting the slot - * @param rmLeaderId current leader id of the ResourceManager - * @throws SlotAllocationException if the slot allocation fails - * @return answer to the slot request - */ @Override public CompletableFuture requestSlot( final SlotID slotId, final JobID jobId, final AllocationID allocationId, final String targetAddress, - final UUID rmLeaderId, + final ResourceManagerId resourceManagerId, final Time timeout) { // TODO: Filter invalid requests from the resource manager by using the instance/registration Id log.info("Receive slot request {} for job {} from resource manager with leader id {}.", - allocationId, jobId, rmLeaderId); + allocationId, jobId, resourceManagerId); try { if (resourceManagerConnection == null) { @@ -616,8 +606,8 @@ public CompletableFuture requestSlot( throw new SlotAllocationException(message); } - if (!resourceManagerConnection.getTargetLeaderId().equals(rmLeaderId)) { - final String message = "The leader id " + rmLeaderId + + if (!Objects.equals(resourceManagerConnection.getTargetLeaderId(), resourceManagerId)) { + final String message = "The leader id " + resourceManagerId + " does not match with the leader id of the connected resource manager " + resourceManagerConnection.getTargetLeaderId() + '.'; @@ -692,7 +682,7 @@ public void disconnectResourceManager(Exception cause) { // Internal resource manager connection methods // ------------------------------------------------------------------------ - private void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) { + private void notifyOfNewResourceManagerLeader(String newLeaderAddress, ResourceManagerId newResourceManagerId) { if (resourceManagerConnection != null) { if (newLeaderAddress != null) { // the resource manager switched to a new leader @@ -723,7 +713,7 @@ private void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newL getResourceID(), taskSlotTable.createSlotReport(getResourceID()), newLeaderAddress, - newLeaderId, + newResourceManagerId, getMainThreadExecutor(), new ResourceManagerRegistrationListener()); resourceManagerConnection.start(); @@ -1079,7 +1069,6 @@ private void freeSlot(AllocationID allocationId, Throwable cause) { ResourceManagerGateway resourceManagerGateway = resourceManagerConnection.getTargetGateway(); resourceManagerGateway.notifySlotAvailable( - resourceManagerConnection.getTargetLeaderId(), resourceManagerConnection.getRegistrationId(), new SlotID(getResourceID(), freedSlotIndex), allocationId); @@ -1169,12 +1158,10 @@ private final class ResourceManagerLeaderListener implements LeaderRetrievalList @Override public void notifyLeaderAddress(final String leaderAddress, final UUID leaderSessionID) { - runAsync(new Runnable() { - @Override - public void run() { - notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID); - } - }); + runAsync( + () -> notifyOfNewResourceManagerLeader( + leaderAddress, + leaderSessionID != null ? new ResourceManagerId(leaderSessionID) : null)); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index 80841545f1059..fd5625550296e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.taskmanager.Task; @@ -44,8 +45,11 @@ public interface TaskExecutorGateway extends RpcGateway { * Requests a slot from the TaskManager * * @param slotId slot id for the request + * @param jobId for which to request a slot * @param allocationId id for the request - * @param resourceManagerLeaderId current leader id of the ResourceManager + * @param targetAddress to which to offer the requested slots + * @param resourceManagerId current leader id of the ResourceManager + * @param timeout for the operation * @return answer to the slot request */ CompletableFuture requestSlot( @@ -53,7 +57,7 @@ CompletableFuture requestSlot( JobID jobId, AllocationID allocationId, String targetAddress, - UUID resourceManagerLeaderId, + ResourceManagerId resourceManagerId, @RpcTimeout Time timeout); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java index 24eb540b76dda..c3d35326399a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java @@ -23,6 +23,7 @@ import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.registration.RegisteredRpcConnection; import org.apache.flink.runtime.registration.RegistrationConnectionListener; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.registration.RetryingRegistration; @@ -31,7 +32,6 @@ import org.apache.flink.util.Preconditions; import org.slf4j.Logger; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -41,7 +41,7 @@ * The connection between a TaskExecutor and the ResourceManager. */ public class TaskExecutorToResourceManagerConnection - extends RegisteredRpcConnection { + extends RegisteredRpcConnection { private final RpcService rpcService; @@ -64,11 +64,11 @@ public TaskExecutorToResourceManagerConnection( ResourceID taskManagerResourceId, SlotReport slotReport, String resourceManagerAddress, - UUID resourceManagerLeaderId, + ResourceManagerId resourceManagerId, Executor executor, RegistrationConnectionListener registrationListener) { - super(log, resourceManagerAddress, resourceManagerLeaderId, executor); + super(log, resourceManagerAddress, resourceManagerId, executor); this.rpcService = Preconditions.checkNotNull(rpcService); this.taskManagerAddress = Preconditions.checkNotNull(taskManagerAddress); @@ -79,7 +79,7 @@ public TaskExecutorToResourceManagerConnection( @Override - protected RetryingRegistration generateRegistration() { + protected RetryingRegistration generateRegistration() { return new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration( log, rpcService, @@ -127,7 +127,7 @@ public ResourceID getResourceManagerId() { // ------------------------------------------------------------------------ private static class ResourceManagerRegistration - extends RetryingRegistration { + extends RetryingRegistration { private final String taskExecutorAddress; @@ -139,12 +139,12 @@ private static class ResourceManagerRegistration Logger log, RpcService rpcService, String targetAddress, - UUID leaderId, + ResourceManagerId resourceManagerId, String taskExecutorAddress, ResourceID resourceID, SlotReport slotReport) { - super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId); + super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, resourceManagerId); this.taskExecutorAddress = checkNotNull(taskExecutorAddress); this.resourceID = checkNotNull(resourceID); this.slotReport = checkNotNull(slotReport); @@ -152,10 +152,10 @@ private static class ResourceManagerRegistration @Override protected CompletableFuture invokeRegistration( - ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception { + ResourceManagerGateway resourceManager, ResourceManagerId fencingToken, long timeoutMillis) throws Exception { Time timeout = Time.milliseconds(timeoutMillis); - return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, slotReport, timeout); + return resourceManager.registerTaskExecutor(taskExecutorAddress, resourceID, slotReport, timeout); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java index 737cede83d84e..55499f5aae025 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java @@ -49,6 +49,7 @@ import org.apache.flink.runtime.resourcemanager.JobLeaderIdService; import org.apache.flink.runtime.resourcemanager.ResourceManagerConfiguration; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManager; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.TestingRpcService; @@ -532,7 +533,6 @@ public void testHeartbeatTimeoutWithTaskExecutor() throws Exception { final SlotReport slotReport = new SlotReport(); // test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time CompletableFuture successfulFuture = rmGateway.registerTaskExecutor( - rmLeaderSessionId, taskManagerAddress, taskManagerResourceID, slotReport, @@ -576,7 +576,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final String jobMasterAddress = "jm"; final ResourceID jmResourceId = new ResourceID(jobMasterAddress); final ResourceID rmResourceId = ResourceID.generate(); - final UUID rmLeaderId = UUID.randomUUID(); + final ResourceManagerId rmLeaderId = ResourceManagerId.generate(); final UUID jmLeaderId = UUID.randomUUID(); final JobID jobId = new JobID(); @@ -629,11 +629,10 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); - rmLeaderElectionService.isLeader(rmLeaderId).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + rmLeaderElectionService.isLeader(rmLeaderId.toUUID()).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); // test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time CompletableFuture successfulFuture = rmGateway.registerJobManager( - rmLeaderId, jmLeaderId, jmResourceId, jobMasterAddress, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java index 68c43f8af9e76..ead453e485e33 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java @@ -88,7 +88,7 @@ public void testAllocateSimpleSlot() throws Exception { assertFalse(future.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); @@ -125,7 +125,7 @@ public void testAllocationFulfilledByReturnedSlot() throws Exception { ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds()).times(2)) - .requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + .requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final List slotRequests = slotRequestArgumentCaptor.getAllValues(); @@ -168,7 +168,7 @@ public void testAllocateWithFreeSlot() throws Exception { assertFalse(future1.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); @@ -211,7 +211,7 @@ public void testOfferSlot() throws Exception { assertFalse(future.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); @@ -266,7 +266,7 @@ public void returnAllocatedSlot(Slot slot) { CompletableFuture future1 = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); @@ -297,7 +297,7 @@ public void returnAllocatedSlot(Slot slot) { private static ResourceManagerGateway createResourceManagerGatewayMock() { ResourceManagerGateway resourceManagerGateway = mock(ResourceManagerGateway.class); when(resourceManagerGateway - .requestSlot(any(UUID.class), any(UUID.class), any(SlotRequest.class), any(Time.class))) + .requestSlot(any(UUID.class), any(SlotRequest.class), any(Time.class))) .thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); return resourceManagerGateway; @@ -310,7 +310,7 @@ private static SlotPoolGateway setupSlotPool( slotPool.start(UUID.randomUUID(), jobManagerAddress); - slotPool.connectToResourceManager(UUID.randomUUID(), resourceManagerGateway); + slotPool.connectToResourceManager(resourceManagerGateway); return slotPool.getSelfGateway(SlotPoolGateway.class); } 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 76a0b93104c03..6282ea0320575 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 @@ -37,6 +37,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; @@ -167,7 +168,7 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { public void testHeartbeatTimeoutWithResourceManager() throws Exception { final String resourceManagerAddress = "rm"; final String jobManagerAddress = "jm"; - final UUID rmLeaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final UUID jmLeaderId = UUID.randomUUID(); final ResourceID rmResourceId = new ResourceID(resourceManagerAddress); final ResourceID jmResourceId = new ResourceID(jobManagerAddress); @@ -187,14 +188,13 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { final ResourceManagerGateway resourceManagerGateway = mock(ResourceManagerGateway.class); when(resourceManagerGateway.registerJobManager( - any(UUID.class), any(UUID.class), any(ResourceID.class), anyString(), any(JobID.class), any(Time.class) )).thenReturn(CompletableFuture.completedFuture(new JobMasterRegistrationSuccess( - heartbeatInterval, rmLeaderId, rmResourceId))); + heartbeatInterval, resourceManagerId, rmResourceId))); final TestingRpcService rpc = new TestingRpcService(); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); @@ -225,11 +225,10 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { startFuture.get(testingTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); // define a leader and see that a registration happens - rmLeaderRetrievalService.notifyListener(resourceManagerAddress, rmLeaderId); + rmLeaderRetrievalService.notifyListener(resourceManagerAddress, resourceManagerId.toUUID()); // register job manager success will trigger monitor heartbeat target between jm and rm verify(resourceManagerGateway, timeout(testingTimeout.toMilliseconds())).registerJobManager( - eq(rmLeaderId), eq(jmLeaderId), eq(jmResourceId), anyString(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java index 5568467452771..ccbb4f4d9a443 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java @@ -205,6 +205,10 @@ public void testReporterScheduling() throws InterruptedException { MetricRegistry registry = new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(config)); long start = System.currentTimeMillis(); + + // only start counting from now on + TestReporter3.reportCount = 0; + for (int x = 0; x < 10; x++) { Thread.sleep(100); int reportCount = TestReporter3.reportCount; @@ -218,7 +222,7 @@ public void testReporterScheduling() throws InterruptedException { * or after T=50. */ long maxAllowedReports = (curT - start) / 50 + 2; - Assert.assertTrue("Too many report were triggered.", maxAllowedReports >= reportCount); + Assert.assertTrue("Too many reports were triggered.", maxAllowedReports >= reportCount); } Assert.assertTrue("No report was triggered.", TestReporter3.reportCount > 0); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index c213752081de3..2b8792b2eba6f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -82,7 +82,7 @@ public void confirmLeaderSessionID(UUID leaderId) { TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - CompletableFuture revokedLeaderIdFuture = new CompletableFuture<>(); + CompletableFuture revokedLeaderIdFuture = new CompletableFuture<>(); final ResourceManager resourceManager = new StandaloneResourceManager( @@ -100,23 +100,28 @@ public void confirmLeaderSessionID(UUID leaderId) { @Override public void revokeLeadership() { super.revokeLeadership(); - runAsync( - () -> revokedLeaderIdFuture.complete(getLeaderSessionId())); + runAsyncWithoutFencing( + () -> revokedLeaderIdFuture.complete(getFencingToken())); } }; - resourceManager.start(); - // before grant leadership, resourceManager's leaderId is null - Assert.assertEquals(null, resourceManager.getLeaderSessionId()); - final UUID leaderId = UUID.randomUUID(); - leaderElectionService.isLeader(leaderId); - // after grant leadership, resourceManager's leaderId has value - Assert.assertEquals(leaderId, leaderSessionIdFuture.get()); - // then revoke leadership, resourceManager's leaderId is null again - leaderElectionService.notLeader(); - Assert.assertEquals(null, revokedLeaderIdFuture.get()); - - if (testingFatalErrorHandler.hasExceptionOccurred()) { - testingFatalErrorHandler.rethrowError(); + + try { + resourceManager.start(); + + Assert.assertNotNull(resourceManager.getFencingToken()); + final UUID leaderId = UUID.randomUUID(); + leaderElectionService.isLeader(leaderId); + // after grant leadership, resourceManager's leaderId has value + Assert.assertEquals(leaderId, leaderSessionIdFuture.get()); + // then revoke leadership, resourceManager's leaderId should be different + leaderElectionService.notLeader(); + Assert.assertNotEquals(leaderId, revokedLeaderIdFuture.get()); + + if (testingFatalErrorHandler.hasExceptionOccurred()) { + testingFatalErrorHandler.rethrowError(); + } + } finally { + rpcService.stopService(); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java index 139bfc4e19e58..1de32841ec723 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -27,15 +27,20 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.rpc.exceptions.FencingTokenMismatchException; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; @@ -43,9 +48,11 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Mockito.*; public class ResourceManagerJobMasterTest extends TestLogger { @@ -71,18 +78,15 @@ public void teardown() throws Exception { public void testRegisterJobMaster() throws Exception { String jobMasterAddress = "/jobMasterAddress1"; JobID jobID = mockJobMaster(jobMasterAddress); - TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); UUID jmLeaderID = UUID.randomUUID(); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManager resourceManager = createAndStartResourceManager(mock(LeaderElectionService.class), jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); - final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); // test response successful CompletableFuture successfulFuture = rmGateway.registerJobManager( - rmLeaderSessionId, jmLeaderID, jmResourceId, jobMasterAddress, @@ -103,25 +107,28 @@ public void testRegisterJobMaster() throws Exception { public void testRegisterJobMasterWithUnmatchedLeaderSessionId1() throws Exception { String jobMasterAddress = "/jobMasterAddress1"; JobID jobID = mockJobMaster(jobMasterAddress); - TestingLeaderElectionService resourceManagerLeaderElectionService = new TestingLeaderElectionService(); UUID jmLeaderID = UUID.randomUUID(); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); - final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); - final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); + final ResourceManager resourceManager = createAndStartResourceManager(mock(LeaderElectionService.class), jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); + final ResourceManagerGateway wronglyFencedGateway = rpcService.connect(resourceManager.getAddress(), ResourceManagerId.generate(), ResourceManagerGateway.class) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); // test throw exception when receive a registration from job master which takes unmatched leaderSessionId - UUID differentLeaderSessionID = UUID.randomUUID(); - CompletableFuture unMatchedLeaderFuture = rmGateway.registerJobManager( - differentLeaderSessionID, + CompletableFuture unMatchedLeaderFuture = wronglyFencedGateway.registerJobManager( jmLeaderID, jmResourceId, jobMasterAddress, jobID, timeout); - assertTrue(unMatchedLeaderFuture.get(5, TimeUnit.SECONDS) instanceof RegistrationResponse.Decline); + + try { + unMatchedLeaderFuture.get(5L, TimeUnit.SECONDS); + fail("Should fail because we are using the wrong fencing token."); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.stripExecutionException(e) instanceof FencingTokenMismatchException); + } if (testingFatalErrorHandler.hasExceptionOccurred()) { testingFatalErrorHandler.rethrowError(); @@ -142,14 +149,11 @@ public void testRegisterJobMasterWithUnmatchedLeaderSessionId2() throws Exceptio TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); - final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); - final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); // test throw exception when receive a registration from job master which takes unmatched leaderSessionId UUID differentLeaderSessionID = UUID.randomUUID(); CompletableFuture unMatchedLeaderFuture = rmGateway.registerJobManager( - rmLeaderSessionId, differentLeaderSessionID, jmResourceId, jobMasterAddress, @@ -176,15 +180,12 @@ public void testRegisterJobMasterFromInvalidAddress() throws Exception { TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final ResourceManager resourceManager = createAndStartResourceManager(resourceManagerLeaderElectionService, jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); - final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); - final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); // test throw exception when receive a registration from job master which takes invalid address String invalidAddress = "/jobMasterAddress2"; CompletableFuture invalidAddressFuture = rmGateway.registerJobManager( - rmLeaderSessionId, - jmLeaderSessionId, + HighAvailabilityServices.DEFAULT_LEADER_ID, jmResourceId, invalidAddress, jobID, @@ -214,24 +215,22 @@ public void testRegisterJobMasterWithFailureLeaderListener() throws Exception { jobMasterLeaderRetrievalService, testingFatalErrorHandler); final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); - final UUID rmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); - final UUID jmLeaderSessionId = grantResourceManagerLeadership(resourceManagerLeaderElectionService); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); JobID unknownJobIDToHAServices = new JobID(); - // verify return RegistrationResponse.Decline when failed to start a job master Leader retrieval listener - CompletableFuture declineFuture = rmGateway.registerJobManager( - rmLeaderSessionId, - jmLeaderSessionId, + + // this should fail because we try to register a job leader listener for an unknown job id + CompletableFuture registrationFuture = rmGateway.registerJobManager( + HighAvailabilityServices.DEFAULT_LEADER_ID, jmResourceId, jobMasterAddress, unknownJobIDToHAServices, timeout); - RegistrationResponse response = declineFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); - assertTrue(response instanceof RegistrationResponse.Decline); - if (testingFatalErrorHandler.hasExceptionOccurred()) { - testingFatalErrorHandler.rethrowError(); + try { + registrationFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.stripExecutionException(e) instanceof ResourceManagerException); } } @@ -243,9 +242,9 @@ private JobID mockJobMaster(String jobMasterAddress) { } private ResourceManager createAndStartResourceManager( - TestingLeaderElectionService resourceManagerLeaderElectionService, + LeaderElectionService resourceManagerLeaderElectionService, JobID jobID, - TestingLeaderRetrievalService jobMasterLeaderRetrievalService, + LeaderRetrievalService jobMasterLeaderRetrievalService, FatalErrorHandler fatalErrorHandler) throws Exception { ResourceID rmResourceId = ResourceID.generate(); TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); @@ -283,11 +282,4 @@ private ResourceManager createAndStartResourceManager( resourceManager.start(); return resourceManager; } - - private UUID grantResourceManagerLeadership(TestingLeaderElectionService resourceManagerLeaderElectionService) { - UUID leaderSessionId = UUID.randomUUID(); - resourceManagerLeaderElectionService.isLeader(leaderSessionId); - return leaderSessionId; - } - } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index 616ed5c985408..8add1685dbea7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -23,17 +23,20 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.slotmanager.SlotManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.registration.RegistrationResponse; +import org.apache.flink.runtime.rpc.exceptions.FencingTokenMismatchException; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; @@ -41,10 +44,12 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; public class ResourceManagerTaskExecutorTest extends TestLogger { @@ -65,7 +70,7 @@ public class ResourceManagerTaskExecutorTest extends TestLogger { private ResourceManagerGateway rmGateway; - private UUID leaderSessionId; + private ResourceManagerGateway wronglyFencedGateway; private TestingFatalErrorHandler testingFatalErrorHandler; @@ -75,11 +80,14 @@ public void setup() throws Exception { taskExecutorResourceID = mockTaskExecutor(taskExecutorAddress); resourceManagerResourceID = ResourceID.generate(); - TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService(); testingFatalErrorHandler = new TestingFatalErrorHandler(); + TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService(); resourceManager = createAndStartResourceManager(rmLeaderElectionService, testingFatalErrorHandler); rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); - leaderSessionId = grantLeadership(rmLeaderElectionService); + wronglyFencedGateway = rpcService.connect(resourceManager.getAddress(), ResourceManagerId.generate(), ResourceManagerGateway.class) + .get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + + grantLeadership(rmLeaderElectionService).get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); } @After @@ -95,13 +103,13 @@ public void testRegisterTaskExecutor() throws Exception { try { // test response successful CompletableFuture successfulFuture = - rmGateway.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); + rmGateway.registerTaskExecutor(taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); RegistrationResponse response = successfulFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); assertTrue(response instanceof TaskExecutorRegistrationSuccess); // test response successful with instanceID not equal to previous when receive duplicate registration from taskExecutor CompletableFuture duplicateFuture = - rmGateway.registerTaskExecutor(leaderSessionId, taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); + rmGateway.registerTaskExecutor(taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); RegistrationResponse duplicateResponse = duplicateFuture.get(); assertTrue(duplicateResponse instanceof TaskExecutorRegistrationSuccess); assertNotEquals(((TaskExecutorRegistrationSuccess) response).getRegistrationId(), ((TaskExecutorRegistrationSuccess) duplicateResponse).getRegistrationId()); @@ -119,10 +127,15 @@ public void testRegisterTaskExecutor() throws Exception { public void testRegisterTaskExecutorWithUnmatchedLeaderSessionId() throws Exception { try { // test throw exception when receive a registration from taskExecutor which takes unmatched leaderSessionId - UUID differentLeaderSessionID = UUID.randomUUID(); CompletableFuture unMatchedLeaderFuture = - rmGateway.registerTaskExecutor(differentLeaderSessionID, taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); - assertTrue(unMatchedLeaderFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS) instanceof RegistrationResponse.Decline); + wronglyFencedGateway.registerTaskExecutor(taskExecutorAddress, taskExecutorResourceID, slotReport, timeout); + + try { + unMatchedLeaderFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS); + fail("Should have failed because we are using a wrongly fenced ResourceManagerGateway."); + } catch (ExecutionException e) { + assertTrue(ExceptionUtils.stripExecutionException(e) instanceof FencingTokenMismatchException); + } } finally { if (testingFatalErrorHandler.hasExceptionOccurred()) { testingFatalErrorHandler.rethrowError(); @@ -139,7 +152,7 @@ public void testRegisterTaskExecutorFromInvalidAddress() throws Exception { // test throw exception when receive a registration from taskExecutor which takes invalid address String invalidAddress = "/taskExecutor2"; CompletableFuture invalidAddressFuture = - rmGateway.registerTaskExecutor(leaderSessionId, invalidAddress, taskExecutorResourceID, slotReport, timeout); + rmGateway.registerTaskExecutor(invalidAddress, taskExecutorResourceID, slotReport, timeout); assertTrue(invalidAddressFuture.get(timeout.toMilliseconds(), TimeUnit.MILLISECONDS) instanceof RegistrationResponse.Decline); } finally { if (testingFatalErrorHandler.hasExceptionOccurred()) { @@ -155,7 +168,7 @@ private ResourceID mockTaskExecutor(String taskExecutorAddress) { return taskExecutorResourceID; } - private StandaloneResourceManager createAndStartResourceManager(TestingLeaderElectionService rmLeaderElectionService, FatalErrorHandler fatalErrorHandler) throws Exception { + private StandaloneResourceManager createAndStartResourceManager(LeaderElectionService rmLeaderElectionService, FatalErrorHandler fatalErrorHandler) throws Exception { TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); HeartbeatServices heartbeatServices = new HeartbeatServices(5L, 5L); highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService); @@ -187,14 +200,15 @@ private StandaloneResourceManager createAndStartResourceManager(TestingLeaderEle metricRegistry, jobLeaderIdService, fatalErrorHandler); + resourceManager.start(); + return resourceManager; } - private UUID grantLeadership(TestingLeaderElectionService leaderElectionService) { + private CompletableFuture grantLeadership(TestingLeaderElectionService leaderElectionService) { UUID leaderSessionId = UUID.randomUUID(); - leaderElectionService.isLeader(leaderSessionId); - return leaderSessionId; + return leaderElectionService.isLeader(leaderSessionId); } } 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 93e96a7b0f026..80b445fbb2b9e 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 @@ -30,6 +30,7 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.exceptions.ResourceManagerException; import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection; @@ -43,7 +44,6 @@ import org.mockito.ArgumentCaptor; import java.util.Arrays; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; @@ -74,7 +74,7 @@ public class SlotManagerTest extends TestLogger { */ @Test public void testTaskManagerRegistration() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); @@ -88,7 +88,7 @@ public void testTaskManagerRegistration() throws Exception { final SlotStatus slotStatus2 = new SlotStatus(slotId2, resourceProfile); final SlotReport slotReport = new SlotReport(Arrays.asList(slotStatus1, slotStatus2)); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerTaskManager(taskManagerConnection, slotReport); assertTrue("The number registered slots does not equal the expected number.",2 == slotManager.getNumberRegisteredSlots()); @@ -103,7 +103,7 @@ public void testTaskManagerRegistration() throws Exception { */ @Test public void testTaskManagerUnregistration() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final JobID jobId = new JobID(); @@ -113,7 +113,7 @@ public void testTaskManagerUnregistration() throws Exception { any(JobID.class), any(AllocationID.class), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class))).thenReturn(new CompletableFuture<>()); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -134,7 +134,7 @@ public void testTaskManagerUnregistration() throws Exception { resourceProfile, "foobar"); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerTaskManager(taskManagerConnection, slotReport); assertTrue("The number registered slots does not equal the expected number.",2 == slotManager.getNumberRegisteredSlots()); @@ -166,7 +166,7 @@ public void testTaskManagerUnregistration() throws Exception { */ @Test public void testSlotRequestWithoutFreeSlots() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceProfile resourceProfile = new ResourceProfile(42.0, 1337); final SlotRequest slotRequest = new SlotRequest( new JobID(), @@ -176,7 +176,7 @@ public void testSlotRequestWithoutFreeSlots() throws Exception { ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerSlotRequest(slotRequest); @@ -189,7 +189,7 @@ public void testSlotRequestWithoutFreeSlots() throws Exception { */ @Test public void testSlotRequestWithResourceAllocationFailure() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceProfile resourceProfile = new ResourceProfile(42.0, 1337); final SlotRequest slotRequest = new SlotRequest( new JobID(), @@ -200,7 +200,7 @@ public void testSlotRequestWithResourceAllocationFailure() throws Exception { ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); doThrow(new ResourceManagerException("Test exception")).when(resourceManagerActions).allocateResource(any(ResourceProfile.class)); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerSlotRequest(slotRequest); @@ -216,7 +216,7 @@ public void testSlotRequestWithResourceAllocationFailure() throws Exception { */ @Test public void testSlotRequestWithFreeSlot() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceID resourceID = ResourceID.generate(); final JobID jobId = new JobID(); final SlotID slotId = new SlotID(resourceID, 0); @@ -231,7 +231,7 @@ public void testSlotRequestWithFreeSlot() throws Exception { ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { // accept an incoming slot request final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); @@ -240,7 +240,7 @@ public void testSlotRequestWithFreeSlot() throws Exception { eq(jobId), eq(allocationId), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -254,7 +254,7 @@ public void testSlotRequestWithFreeSlot() throws Exception { assertTrue("The slot request should be accepted", slotManager.registerSlotRequest(slotRequest)); - verify(taskExecutorGateway).requestSlot(eq(slotId), eq(jobId), eq(allocationId), eq(targetAddress), eq(leaderId), any(Time.class)); + verify(taskExecutorGateway).requestSlot(eq(slotId), eq(jobId), eq(allocationId), eq(targetAddress), eq(resourceManagerId), any(Time.class)); TaskManagerSlot slot = slotManager.getSlot(slotId); @@ -268,7 +268,7 @@ public void testSlotRequestWithFreeSlot() throws Exception { */ @Test public void testUnregisterPendingSlotRequest() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final SlotID slotId = new SlotID(ResourceID.generate(), 0); final AllocationID allocationId = new AllocationID(); @@ -279,7 +279,7 @@ public void testUnregisterPendingSlotRequest() throws Exception { any(JobID.class), any(AllocationID.class), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class))).thenReturn(new CompletableFuture<>()); final ResourceProfile resourceProfile = new ResourceProfile(1.0, 1); @@ -290,7 +290,7 @@ public void testUnregisterPendingSlotRequest() throws Exception { final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerTaskManager(taskManagerConnection, slotReport); TaskManagerSlot slot = slotManager.getSlot(slotId); @@ -315,7 +315,7 @@ public void testUnregisterPendingSlotRequest() throws Exception { */ @Test public void testFulfillingPendingSlotRequest() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceID resourceID = ResourceID.generate(); final JobID jobId = new JobID(); final SlotID slotId = new SlotID(resourceID, 0); @@ -337,7 +337,7 @@ public void testFulfillingPendingSlotRequest() throws Exception { eq(jobId), eq(allocationId), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskExecutorConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -345,7 +345,7 @@ public void testFulfillingPendingSlotRequest() throws Exception { final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile); final SlotReport slotReport = new SlotReport(slotStatus); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { assertTrue("The slot request should be accepted", slotManager.registerSlotRequest(slotRequest)); @@ -355,7 +355,7 @@ public void testFulfillingPendingSlotRequest() throws Exception { taskExecutorConnection, slotReport); - verify(taskExecutorGateway).requestSlot(eq(slotId), eq(jobId), eq(allocationId), eq(targetAddress), eq(leaderId), any(Time.class)); + verify(taskExecutorGateway).requestSlot(eq(slotId), eq(jobId), eq(allocationId), eq(targetAddress), eq(resourceManagerId), any(Time.class)); TaskManagerSlot slot = slotManager.getSlot(slotId); @@ -368,7 +368,7 @@ public void testFulfillingPendingSlotRequest() throws Exception { */ @Test public void testFreeSlot() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceID resourceID = ResourceID.generate(); final JobID jobId = new JobID(); final SlotID slotId = new SlotID(resourceID, 0); @@ -385,7 +385,7 @@ public void testFreeSlot() throws Exception { final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile, jobId, allocationId); final SlotReport slotReport = new SlotReport(slotStatus); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerTaskManager( taskExecutorConnection, @@ -414,8 +414,7 @@ public void testFreeSlot() throws Exception { */ @Test public void testDuplicatePendingSlotRequest() throws Exception { - - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final AllocationID allocationId = new AllocationID(); final ResourceProfile resourceProfile1 = new ResourceProfile(1.0, 2); @@ -423,7 +422,7 @@ public void testDuplicatePendingSlotRequest() throws Exception { final SlotRequest slotRequest1 = new SlotRequest(new JobID(), allocationId, resourceProfile1, "foobar"); final SlotRequest slotRequest2 = new SlotRequest(new JobID(), allocationId, resourceProfile2, "barfoo"); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { assertTrue(slotManager.registerSlotRequest(slotRequest1)); assertFalse(slotManager.registerSlotRequest(slotRequest2)); } @@ -439,7 +438,7 @@ public void testDuplicatePendingSlotRequest() throws Exception { */ @Test public void testDuplicatePendingSlotRequestAfterSlotReport() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final JobID jobId = new JobID(); final AllocationID allocationId = new AllocationID(); @@ -454,7 +453,7 @@ public void testDuplicatePendingSlotRequestAfterSlotReport() throws Exception { final SlotRequest slotRequest = new SlotRequest(jobId, allocationId, resourceProfile, "foobar"); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerTaskManager(taskManagerConnection, slotReport); assertFalse(slotManager.registerSlotRequest(slotRequest)); @@ -467,7 +466,7 @@ public void testDuplicatePendingSlotRequestAfterSlotReport() throws Exception { */ @Test public void testDuplicatePendingSlotRequestAfterSuccessfulAllocation() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final AllocationID allocationId = new AllocationID(); final ResourceProfile resourceProfile1 = new ResourceProfile(1.0, 2); @@ -481,7 +480,7 @@ public void testDuplicatePendingSlotRequestAfterSuccessfulAllocation() throws Ex any(JobID.class), any(AllocationID.class), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -490,7 +489,7 @@ public void testDuplicatePendingSlotRequestAfterSuccessfulAllocation() throws Ex final SlotStatus slotStatus = new SlotStatus(slotId, resourceProfile1); final SlotReport slotReport = new SlotReport(slotStatus); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerTaskManager(taskManagerConnection, slotReport); assertTrue(slotManager.registerSlotRequest(slotRequest1)); @@ -512,7 +511,7 @@ public void testDuplicatePendingSlotRequestAfterSuccessfulAllocation() throws Ex */ @Test public void testAcceptingDuplicateSlotRequestAfterAllocationRelease() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final AllocationID allocationId = new AllocationID(); final ResourceProfile resourceProfile1 = new ResourceProfile(1.0, 2); @@ -526,7 +525,7 @@ public void testAcceptingDuplicateSlotRequestAfterAllocationRelease() throws Exc any(JobID.class), any(AllocationID.class), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -535,7 +534,7 @@ public void testAcceptingDuplicateSlotRequestAfterAllocationRelease() throws Exc final SlotStatus slotStatus = new SlotStatus(slotId, new ResourceProfile(2.0, 2)); final SlotReport slotReport = new SlotReport(slotStatus); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerTaskManager(taskManagerConnection, slotReport); assertTrue(slotManager.registerSlotRequest(slotRequest1)); @@ -565,7 +564,7 @@ public void testAcceptingDuplicateSlotRequestAfterAllocationRelease() throws Exc */ @Test public void testReceivingUnknownSlotReport() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final InstanceID unknownInstanceID = new InstanceID(); @@ -574,7 +573,7 @@ public void testReceivingUnknownSlotReport() throws Exception { final SlotStatus unknownSlotStatus = new SlotStatus(unknownSlotId, unknownResourceProfile); final SlotReport unknownSlotReport = new SlotReport(unknownSlotStatus); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { // check that we don't have any slots registered assertTrue(0 == slotManager.getNumberRegisteredSlots()); @@ -591,7 +590,7 @@ public void testReceivingUnknownSlotReport() throws Exception { */ @Test public void testUpdateSlotReport() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final JobID jobId = new JobID(); @@ -614,7 +613,7 @@ public void testUpdateSlotReport() throws Exception { final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { // check that we don't have any slots registered assertTrue(0 == slotManager.getNumberRegisteredSlots()); @@ -651,7 +650,7 @@ public void testTaskManagerTimeout() throws Exception { final long tmTimeout = 500L; final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -669,7 +668,7 @@ public void testTaskManagerTimeout() throws Exception { TestingUtils.infiniteTime(), Time.milliseconds(tmTimeout))) { - slotManager.start(leaderId, mainThreadExecutor, resourceManagerActions); + slotManager.start(resourceManagerId, mainThreadExecutor, resourceManagerActions); mainThreadExecutor.execute(new Runnable() { @Override @@ -693,7 +692,7 @@ public void testSlotRequestTimeout() throws Exception { final long allocationTimeout = 50L; final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final JobID jobId = new JobID(); final AllocationID allocationId = new AllocationID(); @@ -708,7 +707,7 @@ public void testSlotRequestTimeout() throws Exception { Time.milliseconds(allocationTimeout), TestingUtils.infiniteTime())) { - slotManager.start(leaderId, mainThreadExecutor, resourceManagerActions); + slotManager.start(resourceManagerId, mainThreadExecutor, resourceManagerActions); final AtomicReference atomicException = new AtomicReference<>(null); @@ -740,7 +739,7 @@ public void run() { @Test @SuppressWarnings("unchecked") public void testTaskManagerSlotRequestTimeoutHandling() throws Exception { - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final JobID jobId = new JobID(); @@ -756,7 +755,7 @@ public void testTaskManagerSlotRequestTimeoutHandling() throws Exception { any(JobID.class), eq(allocationId), anyString(), - any(UUID.class), + any(ResourceManagerId.class), any(Time.class))).thenReturn(slotRequestFuture1, slotRequestFuture2); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -768,7 +767,7 @@ public void testTaskManagerSlotRequestTimeoutHandling() throws Exception { final SlotStatus slotStatus2 = new SlotStatus(slotId2, resourceProfile); final SlotReport slotReport = new SlotReport(Arrays.asList(slotStatus1, slotStatus2)); - try (SlotManager slotManager = createSlotManager(leaderId, resourceManagerActions)) { + try (SlotManager slotManager = createSlotManager(resourceManagerId, resourceManagerActions)) { slotManager.registerTaskManager(taskManagerConnection, slotReport); @@ -781,7 +780,7 @@ public void testTaskManagerSlotRequestTimeoutHandling() throws Exception { eq(jobId), eq(allocationId), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class)); TaskManagerSlot failedSlot = slotManager.getSlot(slotIdCaptor.getValue()); @@ -794,7 +793,7 @@ public void testTaskManagerSlotRequestTimeoutHandling() throws Exception { eq(jobId), eq(allocationId), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class)); // the second attempt succeeds @@ -819,7 +818,7 @@ public void testTaskManagerSlotRequestTimeoutHandling() throws Exception { @SuppressWarnings("unchecked") public void testSlotReportWhileActiveSlotRequest() throws Exception { final long verifyTimeout = 1000L; - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final JobID jobId = new JobID(); @@ -834,7 +833,7 @@ public void testSlotReportWhileActiveSlotRequest() throws Exception { any(JobID.class), eq(allocationId), anyString(), - any(UUID.class), + any(ResourceManagerId.class), any(Time.class))).thenReturn(slotRequestFuture1, CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -854,7 +853,7 @@ public void testSlotReportWhileActiveSlotRequest() throws Exception { TestingUtils.infiniteTime(), TestingUtils.infiniteTime())) { - slotManager.start(leaderId, mainThreadExecutor, resourceManagerActions); + slotManager.start(resourceManagerId, mainThreadExecutor, resourceManagerActions); CompletableFuture registrationFuture = CompletableFuture.supplyAsync( () -> { @@ -882,7 +881,7 @@ public void testSlotReportWhileActiveSlotRequest() throws Exception { eq(jobId), eq(allocationId), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class)); final SlotID requestedSlotId = slotIdCaptor.getValue(); @@ -908,7 +907,7 @@ public void testSlotReportWhileActiveSlotRequest() throws Exception { eq(jobId), eq(allocationId), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class)); final SlotID requestedSlotId2 = slotIdCaptor.getValue(); @@ -935,7 +934,7 @@ public void testTimeoutForUnusedTaskManager() throws Exception { final long taskManagerTimeout = 50L; final long verifyTimeout = taskManagerTimeout * 10L; - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final ResourceManagerActions resourceManagerActions = mock(ResourceManagerActions.class); final ScheduledExecutor scheduledExecutor = TestingUtils.defaultScheduledExecutor(); @@ -952,7 +951,7 @@ public void testTimeoutForUnusedTaskManager() throws Exception { eq(jobId), eq(allocationId), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); final TaskExecutorConnection taskManagerConnection = new TaskExecutorConnection(taskExecutorGateway); @@ -971,7 +970,7 @@ public void testTimeoutForUnusedTaskManager() throws Exception { TestingUtils.infiniteTime(), Time.of(taskManagerTimeout, TimeUnit.MILLISECONDS))) { - slotManager.start(leaderId, mainThreadExecutor, resourceManagerActions); + slotManager.start(resourceManagerId, mainThreadExecutor, resourceManagerActions); CompletableFuture.supplyAsync( () -> { @@ -991,7 +990,7 @@ public void testTimeoutForUnusedTaskManager() throws Exception { eq(jobId), eq(allocationId), anyString(), - eq(leaderId), + eq(resourceManagerId), any(Time.class)); CompletableFuture idleFuture = CompletableFuture.supplyAsync( @@ -1023,14 +1022,14 @@ public void testTimeoutForUnusedTaskManager() throws Exception { } } - private SlotManager createSlotManager(UUID leaderId, ResourceManagerActions resourceManagerActions) { + private SlotManager createSlotManager(ResourceManagerId resourceManagerId, ResourceManagerActions resourceManagerActions) { SlotManager slotManager = new SlotManager( TestingUtils.defaultScheduledExecutor(), TestingUtils.infiniteTime(), TestingUtils.infiniteTime(), TestingUtils.infiniteTime()); - slotManager.start(leaderId, Executors.directExecutor(), resourceManagerActions); + slotManager.start(resourceManagerId, Executors.directExecutor(), resourceManagerActions); return slotManager; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java index 844e1597f7c94..6de4d52af391a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.resourcemanager.registration.TaskExecutorConnection; import org.apache.flink.runtime.taskexecutor.SlotReport; @@ -39,7 +40,6 @@ import org.mockito.Mockito; import java.util.Collections; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -77,7 +77,7 @@ public static void afterClass() { public void testSlotsUnavailableRequest() throws Exception { final JobID jobID = new JobID(); - final UUID rmLeaderID = UUID.randomUUID(); + final ResourceManagerId rmLeaderID = ResourceManagerId.generate(); try (SlotManager slotManager = new SlotManager( scheduledExecutor, @@ -103,7 +103,7 @@ public void testSlotsUnavailableRequest() throws Exception { TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); Mockito.when( taskExecutorGateway - .requestSlot(any(SlotID.class), any(JobID.class), any(AllocationID.class), any(String.class), any(UUID.class), any(Time.class))) + .requestSlot(any(SlotID.class), any(JobID.class), any(AllocationID.class), any(String.class), any(ResourceManagerId.class), any(Time.class))) .thenReturn(mock(CompletableFuture.class)); final ResourceID resourceID = ResourceID.generate(); @@ -118,7 +118,7 @@ public void testSlotsUnavailableRequest() throws Exception { // 4) Slot becomes available and TaskExecutor gets a SlotRequest verify(taskExecutorGateway, timeout(5000L)) - .requestSlot(eq(slotID), eq(jobID), eq(allocationID), any(String.class), any(UUID.class), any(Time.class)); + .requestSlot(eq(slotID), eq(jobID), eq(allocationID), any(String.class), any(ResourceManagerId.class), any(Time.class)); } } @@ -133,12 +133,12 @@ public void testSlotsUnavailableRequest() throws Exception { public void testSlotAvailableRequest() throws Exception { final JobID jobID = new JobID(); - final UUID rmLeaderID = UUID.randomUUID(); + final ResourceManagerId rmLeaderID = ResourceManagerId.generate(); TaskExecutorGateway taskExecutorGateway = mock(TaskExecutorGateway.class); Mockito.when( taskExecutorGateway - .requestSlot(any(SlotID.class), any(JobID.class), any(AllocationID.class), any(String.class), any(UUID.class), any(Time.class))) + .requestSlot(any(SlotID.class), any(JobID.class), any(AllocationID.class), any(String.class), any(ResourceManagerId.class), any(Time.class))) .thenReturn(mock(CompletableFuture.class)); try (SlotManager slotManager = new SlotManager( @@ -171,7 +171,7 @@ public void testSlotAvailableRequest() throws Exception { // a SlotRequest is routed to the TaskExecutor verify(taskExecutorGateway, timeout(5000)) - .requestSlot(eq(slotID), eq(jobID), eq(allocationID), any(String.class), any(UUID.class), any(Time.class)); + .requestSlot(eq(slotID), eq(jobID), eq(allocationID), any(String.class), any(ResourceManagerId.class), any(Time.class)); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index 90f731dc4ae54..945cbf36ff2cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -193,7 +193,6 @@ public void testSlotAllocation() throws Exception { rmLeaderRetrievalService.notifyListener(rmAddress, rmLeaderId); CompletableFuture registrationResponseFuture = rmGateway.registerJobManager( - rmLeaderId, jmLeaderId, jmResourceId, jmAddress, @@ -204,7 +203,7 @@ public void testSlotAllocation() throws Exception { assertTrue(registrationResponse instanceof JobMasterRegistrationSuccess); - CompletableFuture slotAck = rmGateway.requestSlot(jmLeaderId, rmLeaderId, slotRequest, timeout); + CompletableFuture slotAck = rmGateway.requestSlot(jmLeaderId, slotRequest, timeout); slotAck.get(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 6ab52ed46e4fc..2112c1bd4021a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -65,6 +65,7 @@ import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException; @@ -252,7 +253,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { // register the mock resource manager gateway ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); when(rmGateway.registerTaskExecutor( - any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) + anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn( CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess( @@ -327,7 +328,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation // register resource manager success will trigger monitoring heartbeat target between tm and rm verify(rmGateway, Mockito.timeout(timeout.toMilliseconds()).atLeast(1)).registerTaskExecutor( - eq(rmLeaderId), eq(taskManager.getAddress()), eq(tmResourceId), any(SlotReport.class), any(Time.class)); + eq(taskManager.getAddress()), eq(tmResourceId), any(SlotReport.class), any(Time.class)); // heartbeat timeout should trigger disconnect TaskManager from ResourceManager verify(rmGateway, timeout(heartbeatTimeout * 50L)).disconnectTaskManager(eq(taskManagerLocation.getResourceID()), any(TimeoutException.class)); @@ -356,7 +357,7 @@ public void testHeartbeatSlotReporting() throws Exception { // register the mock resource manager gateway ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); when(rmGateway.registerTaskExecutor( - any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) + anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn( CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess( @@ -448,7 +449,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation // register resource manager success will trigger monitoring heartbeat target between tm and rm verify(rmGateway, timeout(verificationTimeout).atLeast(1)).registerTaskExecutor( - eq(rmLeaderId), eq(taskManager.getAddress()), eq(tmResourceId), eq(slotReport1), any(Time.class)); + eq(taskManager.getAddress()), eq(tmResourceId), eq(slotReport1), any(Time.class)); verify(heartbeatManager, timeout(verificationTimeout)).monitorTarget(any(ResourceID.class), any(HeartbeatTarget.class)); @@ -489,7 +490,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { // register a mock resource manager gateway ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class); when(rmGateway.registerTaskExecutor( - any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) + anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn(CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess( new InstanceID(), resourceManagerResourceId, 10L))); @@ -535,7 +536,7 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { String taskManagerAddress = taskManager.getAddress(); verify(rmGateway, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( - any(UUID.class), eq(taskManagerAddress), eq(resourceID), eq(slotReport), any(Time.class)); + eq(taskManagerAddress), eq(resourceID), eq(slotReport), any(Time.class)); // check if a concurrent error occurred testingFatalErrorHandler.rethrowError(); @@ -562,11 +563,11 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class); when(rmGateway1.registerTaskExecutor( - any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) + anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn(CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId1, 10L))); when(rmGateway2.registerTaskExecutor( - any(UUID.class), anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) + anyString(), any(ResourceID.class), any(SlotReport.class), any(Time.class))) .thenReturn(CompletableFuture.completedFuture( new TaskExecutorRegistrationSuccess(new InstanceID(), rmResourceId2, 10L))); @@ -624,7 +625,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { testLeaderService.notifyListener(address1, leaderId1); verify(rmGateway1, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( - eq(leaderId1), eq(taskManagerAddress), eq(tmResourceID), any(SlotReport.class), any(Time.class)); + eq(taskManagerAddress), eq(tmResourceID), any(SlotReport.class), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); // cancel the leader @@ -634,7 +635,7 @@ public void testTriggerRegistrationOnLeaderChange() throws Exception { testLeaderService.notifyListener(address2, leaderId2); verify(rmGateway2, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( - eq(leaderId2), eq(taskManagerAddress), eq(tmResourceID), eq(slotReport), any(Time.class)); + eq(taskManagerAddress), eq(tmResourceID), eq(slotReport), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); // check if a concurrent error occurred @@ -810,14 +811,13 @@ public void testJobLeaderDetection() throws Exception { haServices.setJobMasterLeaderRetriever(jobId, jobManagerLeaderRetrievalService); final String resourceManagerAddress = "rm"; - final UUID resourceManagerLeaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerLeaderId = ResourceManagerId.generate(); final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress); final ResourceManagerGateway resourceManagerGateway = mock(ResourceManagerGateway.class); final InstanceID registrationId = new InstanceID(); when(resourceManagerGateway.registerTaskExecutor( - eq(resourceManagerLeaderId), any(String.class), eq(resourceId), any(SlotReport.class), @@ -874,7 +874,7 @@ public void testJobLeaderDetection() throws Exception { final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); // tell the task manager about the rm leader - resourceManagerLeaderRetrievalService.notifyListener(resourceManagerAddress, resourceManagerLeaderId); + resourceManagerLeaderRetrievalService.notifyListener(resourceManagerAddress, resourceManagerLeaderId.toUUID()); // request slots from the task manager under the given allocation id CompletableFuture slotRequestAck = tmGateway.requestSlot( @@ -940,7 +940,6 @@ public void testSlotAcceptance() throws Exception { final InstanceID registrationId = new InstanceID(); when(resourceManagerGateway.registerTaskExecutor( - eq(resourceManagerLeaderId), any(String.class), eq(resourceId), any(SlotReport.class), @@ -1000,7 +999,6 @@ public void testSlotAcceptance() throws Exception { jobLeaderService.addJob(jobId, jobManagerAddress); verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).notifySlotAvailable( - eq(resourceManagerLeaderId), eq(registrationId), eq(new SlotID(resourceId, 1)), eq(allocationId2)); @@ -1029,7 +1027,7 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { final ResourceID resourceID = ResourceID.generate(); final String address1 = "/resource/manager/address/one"; - final UUID leaderId = UUID.randomUUID(); + final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); final JobID jobId = new JobID(); final String jobManagerAddress = "foobar"; @@ -1055,7 +1053,7 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { when(taskSlotTable.createSlotReport(any(ResourceID.class))).thenReturn(new SlotReport()); when(taskSlotTable.getCurrentAllocation(1)).thenReturn(new AllocationID()); - when(rmGateway1.registerTaskExecutor(any(UUID.class), anyString(), eq(resourceID), any(SlotReport.class), any(Time.class))).thenReturn( + when(rmGateway1.registerTaskExecutor(anyString(), eq(resourceID), any(SlotReport.class), any(Time.class))).thenReturn( CompletableFuture.completedFuture(new TaskExecutorRegistrationSuccess(new InstanceID(), ResourceID.generate(), 1000L))); TaskExecutor taskManager = new TaskExecutor( @@ -1087,15 +1085,15 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { assertNull(taskManager.getResourceManagerConnection()); // define a leader and see that a registration happens - testLeaderService.notifyListener(address1, leaderId); + testLeaderService.notifyListener(address1, resourceManagerId.toUUID()); verify(rmGateway1, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( - eq(leaderId), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class)); + eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class)); assertNotNull(taskManager.getResourceManagerConnection()); // test that allocating a slot works final SlotID slotID = new SlotID(resourceID, 0); - tmGateway.requestSlot(slotID, jobId, new AllocationID(), jobManagerAddress, leaderId, timeout); + tmGateway.requestSlot(slotID, jobId, new AllocationID(), jobManagerAddress, resourceManagerId, timeout); // TODO: Figure out the concrete allocation behaviour between RM and TM. Maybe we don't need the SlotID... // test that we can't allocate slots which are blacklisted due to pending confirmation of the RM @@ -1106,7 +1104,7 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { jobId, new AllocationID(), jobManagerAddress, - leaderId, + resourceManagerId, timeout); try { @@ -1119,8 +1117,8 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { // re-register verify(rmGateway1, Mockito.timeout(timeout.toMilliseconds())).registerTaskExecutor( - eq(leaderId), eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class)); - testLeaderService.notifyListener(address1, leaderId); + eq(taskManagerAddress), eq(resourceID), any(SlotReport.class), any(Time.class)); + testLeaderService.notifyListener(address1, resourceManagerId.toUUID()); // now we should be successful because the slots status has been synced // test that we can't allocate slots which are blacklisted due to pending confirmation of the RM @@ -1129,7 +1127,7 @@ public void testRejectAllocationRequestsForOutOfSyncSlots() throws Exception { jobId, new AllocationID(), jobManagerAddress, - leaderId, + resourceManagerId, timeout); // check if a concurrent error occurred @@ -1176,7 +1174,6 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { final InstanceID registrationId = new InstanceID(); when(resourceManagerGateway.registerTaskExecutor( - eq(resourceManagerLeaderId), any(String.class), eq(resourceId), any(SlotReport.class), @@ -1317,7 +1314,6 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { offerResultFuture.complete(Collections.singleton(offer1)); verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).notifySlotAvailable( - eq(resourceManagerLeaderId), eq(registrationId), eq(new SlotID(resourceId, 1)), any(AllocationID.class)); From ff1660629fea73886bf9c9f802c80dd9bf84c83d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 26 Aug 2017 11:45:36 +0200 Subject: [PATCH 089/129] [FLINK-7523] Add proper resource shutdown to ResourceManager/JobManagerRunner This commit waits for the completion of the shutdown of the ResourceManager before shutting down the ResourceManagerRuntimeServices. The JobManagerServices are now exclusively passed in to the JobManagerRunner which means that it is no longer responsible for shutting the JobManagerServices down. Additionally, it waits until the JobMaster has been shut down before closing the LeaderElectionService as well as the JobManagerMetricGroup. The JobManagerServices are now managed by the caller of the JobManagerRunner. This allows to reuse them across multiple JobManagerRunners. The RpcEndpoint#postStop method is now called by the UntypedActor#postStop method, which ensures that the RpcEndpoint's method is also called if only the underlying RpcService is shut down (without explicitly shutting down the RpcEndpoint). This closes #4596. --- .../flink/runtime/dispatcher/Dispatcher.java | 45 +++++-- .../dispatcher/StandaloneDispatcher.java | 5 +- .../entrypoint/JobClusterEntrypoint.java | 19 ++- .../runtime/jobmaster/JobManagerRunner.java | 112 +++++------------- .../runtime/minicluster/MiniCluster.java | 2 + .../minicluster/MiniClusterJobDispatcher.java | 23 +++- .../ResourceManagerRunner.java | 49 ++++---- .../flink/runtime/rpc/akka/AkkaRpcActor.java | 100 ++++++---------- .../runtime/rpc/akka/AkkaRpcService.java | 6 +- .../runtime/rpc/messages/ControlMessage.java | 26 ---- .../flink/runtime/rpc/messages/Shutdown.java | 36 ------ .../runtime/dispatcher/DispatcherTest.java | 3 +- 12 files changed, 176 insertions(+), 250 deletions(-) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/ControlMessage.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/Shutdown.java 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 00cbb2fc16140..8977415a8f07d 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 @@ -34,6 +34,7 @@ import org.apache.flink.runtime.jobmanager.SubmittedJobGraph; import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerServices; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; @@ -69,7 +70,7 @@ public abstract class Dispatcher extends FencedRpcEndpoint impleme private final RunningJobsRegistry runningJobsRegistry; private final HighAvailabilityServices highAvailabilityServices; - private final BlobServer blobServer; + private final JobManagerServices jobManagerServices; private final HeartbeatServices heartbeatServices; private final MetricRegistry metricRegistry; @@ -92,7 +93,9 @@ protected Dispatcher( this.configuration = Preconditions.checkNotNull(configuration); this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices); - this.blobServer = Preconditions.checkNotNull(blobServer); + this.jobManagerServices = JobManagerServices.fromConfiguration( + configuration, + Preconditions.checkNotNull(blobServer)); this.heartbeatServices = Preconditions.checkNotNull(heartbeatServices); this.metricRegistry = Preconditions.checkNotNull(metricRegistry); this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler); @@ -111,10 +114,16 @@ protected Dispatcher( @Override public void postStop() throws Exception { - Exception exception = null; + Throwable exception = null; clearState(); + try { + jobManagerServices.shutdown(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + try { submittedJobGraphStore.stop(); } catch (Exception e) { @@ -184,8 +193,8 @@ public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) configuration, getRpcService(), highAvailabilityServices, - blobServer, heartbeatServices, + jobManagerServices, metricRegistry, new DispatcherOnCompleteActions(jobGraph.getJobID()), fatalErrorHandler); @@ -247,13 +256,23 @@ private void removeJob(JobID jobId, boolean cleanupHA) throws Exception { * *

    The state are all currently running jobs. */ - private void clearState() { + private void clearState() throws Exception { + Exception exception = null; + // stop all currently running JobManager since they run in the same process for (JobManagerRunner jobManagerRunner : jobManagerRunners.values()) { - jobManagerRunner.shutdown(); + try { + jobManagerRunner.shutdown(); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); + } } jobManagerRunners.clear(); + + if (exception != null) { + throw exception; + } } /** @@ -296,8 +315,8 @@ protected abstract JobManagerRunner createJobManagerRunner( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - BlobServer blobServer, HeartbeatServices heartbeatServices, + JobManagerServices jobManagerServices, MetricRegistry metricRegistry, OnCompletionActions onCompleteActions, FatalErrorHandler fatalErrorHandler) throws Exception; @@ -321,7 +340,11 @@ public void grantLeadership(final UUID newLeaderSessionID) { // clear the state if we've been the leader before if (getFencingToken() != null) { - clearState(); + try { + clearState(); + } catch (Exception e) { + log.warn("Could not properly clear the Dispatcher state while granting leadership.", e); + } } setFencingToken(dispatcherId); @@ -342,7 +365,11 @@ public void revokeLeadership() { runAsyncWithoutFencing( () -> { log.info("Dispatcher {} was revoked leadership.", getAddress()); - clearState(); + try { + clearState(); + } catch (Exception e) { + log.warn("Could not properly clear the Dispatcher state while revoking leadership.", e); + } setFencingToken(DispatcherId.generate()); }); 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 dfd6a8aac9bf1..d6d82b1bd42b9 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 @@ -26,6 +26,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerServices; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -64,8 +65,8 @@ protected JobManagerRunner createJobManagerRunner( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - BlobServer blobServer, HeartbeatServices heartbeatServices, + JobManagerServices jobManagerServices, MetricRegistry metricRegistry, OnCompletionActions onCompleteActions, FatalErrorHandler fatalErrorHandler) throws Exception { @@ -76,8 +77,8 @@ protected JobManagerRunner createJobManagerRunner( configuration, rpcService, highAvailabilityServices, - blobServer, heartbeatServices, + jobManagerServices, metricRegistry, onCompleteActions, fatalErrorHandler); 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 e70f6c8ca7943..124c6c62fcaba 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 @@ -28,6 +28,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerServices; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.resourcemanager.ResourceManager; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -43,6 +44,8 @@ public abstract class JobClusterEntrypoint extends ClusterEntrypoint { private ResourceManager resourceManager; + private JobManagerServices jobManagerServices; + private JobManagerRunner jobManagerRunner; public JobClusterEntrypoint(Configuration configuration) { @@ -67,12 +70,14 @@ protected void startClusterComponents( metricRegistry, this); + jobManagerServices = JobManagerServices.fromConfiguration(configuration, blobServer); + jobManagerRunner = createJobManagerRunner( configuration, ResourceID.generate(), rpcService, highAvailabilityServices, - blobServer, + jobManagerServices, heartbeatServices, metricRegistry, this); @@ -89,7 +94,7 @@ protected JobManagerRunner createJobManagerRunner( ResourceID resourceId, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - BlobServer blobService, + JobManagerServices jobManagerServices, HeartbeatServices heartbeatServices, MetricRegistry metricRegistry, FatalErrorHandler fatalErrorHandler) throws Exception { @@ -102,8 +107,8 @@ protected JobManagerRunner createJobManagerRunner( configuration, rpcService, highAvailabilityServices, - blobService, heartbeatServices, + jobManagerServices, metricRegistry, new TerminatingOnCompleteActions(jobGraph.getJobID()), fatalErrorHandler); @@ -121,6 +126,14 @@ protected void stopClusterComponents(boolean cleanupHaData) throws Exception { } } + if (jobManagerServices != null) { + try { + jobManagerServices.shutdown(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + } + if (resourceManager != null) { try { resourceManager.shutDown(); 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 8766fabb7a21e..b5b4b826144c2 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 @@ -22,9 +22,9 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; @@ -36,10 +36,10 @@ 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.MetricRegistryConfiguration; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.ExceptionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,53 +92,6 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F // ------------------------------------------------------------------------ - public JobManagerRunner( - final ResourceID resourceId, - final JobGraph jobGraph, - final Configuration configuration, - final RpcService rpcService, - final HighAvailabilityServices haServices, - final BlobServer blobService, - final HeartbeatServices heartbeatServices, - final OnCompletionActions toNotifyOnComplete, - final FatalErrorHandler errorHandler) throws Exception { - this( - resourceId, - jobGraph, - configuration, - rpcService, - haServices, - blobService, - heartbeatServices, - new MetricRegistry(MetricRegistryConfiguration.fromConfiguration(configuration)), - toNotifyOnComplete, - errorHandler); - } - - public JobManagerRunner( - final ResourceID resourceId, - final JobGraph jobGraph, - final Configuration configuration, - final RpcService rpcService, - final HighAvailabilityServices haServices, - final BlobServer blobService, - final HeartbeatServices heartbeatServices, - final MetricRegistry metricRegistry, - final OnCompletionActions toNotifyOnComplete, - final FatalErrorHandler errorHandler) throws Exception { - this( - resourceId, - jobGraph, - configuration, - rpcService, - haServices, - heartbeatServices, - JobManagerServices.fromConfiguration(configuration, blobService), - metricRegistry, - toNotifyOnComplete, - errorHandler); - } - /** * *

    Exceptions that occur while creating the JobManager or JobManagerRunner are directly @@ -217,12 +170,6 @@ public JobManagerRunner( } catch (Throwable t) { // clean up everything - try { - jobManagerServices.shutdown(); - } catch (Throwable tt) { - log.error("Error while shutting down JobManager services", tt); - } - if (jobManagerMetrics != null) { jobManagerMetrics.close(); } @@ -245,40 +192,37 @@ public void start() throws Exception { } } - public void shutdown() { - shutdownInternally(); + public void shutdown() throws Exception { + shutdownInternally().get(); } - private void shutdownInternally() { + private CompletableFuture shutdownInternally() { synchronized (lock) { shutdown = true; - if (leaderElectionService != null) { - try { - leaderElectionService.stop(); - } catch (Throwable t) { - log.error("Could not properly shutdown the leader election service", t); - } - } - - try { - jobManager.shutDown(); - } catch (Throwable t) { - log.error("Error shutting down JobManager", t); - } - - try { - jobManagerServices.shutdown(); - } catch (Throwable t) { - log.error("Error shutting down JobManager services", t); - } - - // make all registered metrics go away - try { - jobManagerMetricGroup.close(); - } catch (Throwable t) { - log.error("Error while unregistering metrics", t); - } + jobManager.shutDown(); + + return jobManager.getTerminationFuture() + .thenAccept( + ignored -> { + Throwable exception = null; + try { + leaderElectionService.stop(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + + // make all registered metrics go away + try { + jobManagerMetricGroup.close(); + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + + if (exception != null) { + throw new FlinkFutureException("Could not properly shut down the JobManagerRunner.", exception); + } + }); } } 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 95f430c0ee245..2fe0587cd4bcc 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 @@ -353,6 +353,8 @@ private void shutdownInternally() throws Exception { if (tm != null) { try { tm.shutDown(); + // wait for the TaskManager to properly terminate + tm.getTerminationFuture().get(); } catch (Throwable t) { exception = firstOrSuppressed(t, exception); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java index 2bb94f2dec142..60d9a6692609a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniClusterJobDispatcher.java @@ -33,6 +33,8 @@ import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -156,7 +158,7 @@ public MiniClusterJobDispatcher( * Shuts down the mini cluster dispatcher. If a job is currently running, that job will be * terminally failed. */ - public void shutdown() { + public void shutdown() throws Exception { synchronized (lock) { if (!shutdown) { shutdown = true; @@ -166,14 +168,31 @@ public void shutdown() { // in this shutdown code we copy the references to the stack first, // to avoid concurrent modification + Throwable exception = null; + JobManagerRunner[] runners = this.runners; if (runners != null) { this.runners = null; for (JobManagerRunner runner : runners) { - runner.shutdown(); + try { + runner.shutdown(); + } catch (Throwable e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); + } } } + + // shut down the JobManagerServices + try { + jobManagerServices.shutdown(); + } catch (Throwable throwable) { + exception = ExceptionUtils.firstOrSuppressed(throwable, exception); + } + + if (exception != null) { + throw new FlinkException("Could not properly terminate all JobManagerRunners.", exception); + } } } } 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 d0c411ceea6bb..ed6e18c60adff 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,16 +20,18 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.concurrent.FlinkFutureException; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.concurrent.CompletableFuture; + /** * Simple {@link StandaloneResourceManager} runner. It instantiates the resource manager's services * and handles fatal errors by shutting the resource manager down. @@ -91,27 +93,23 @@ public void start() throws Exception { } public void shutDown() throws Exception { - shutDownInternally(); + // wait for the completion + shutDownInternally().get(); } - private void shutDownInternally() throws Exception { - Exception exception = null; + private CompletableFuture shutDownInternally() { synchronized (lock) { - try { - resourceManager.shutDown(); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - - try { - resourceManagerRuntimeServices.shutDown(); - } catch (Exception e) { - exception = ExceptionUtils.firstOrSuppressed(e, exception); - } - - if (exception != null) { - ExceptionUtils.rethrow(exception, "Error while shutting down the resource manager runner."); - } + resourceManager.shutDown(); + + return resourceManager.getTerminationFuture() + .thenAccept( + ignored -> { + try { + resourceManagerRuntimeServices.shutDown(); + } catch (Exception e) { + throw new FlinkFutureException("Could not properly shut down the resource manager runtime services.", e); + } + }); } } @@ -123,10 +121,13 @@ private void shutDownInternally() throws Exception { public void onFatalError(Throwable exception) { LOG.error("Encountered fatal error.", exception); - try { - shutDownInternally(); - } catch (Exception e) { - LOG.error("Could not properly shut down the resource manager.", e); - } + CompletableFuture shutdownFuture = shutDownInternally(); + + shutdownFuture.whenComplete( + (Void ignored, Throwable throwable) -> { + if (throwable != null) { + LOG.error("Could not properly shut down the resource manager runner.", throwable); + } + }); } } 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 74c1509757a32..f6c2e8be6faf3 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 @@ -18,31 +18,26 @@ package org.apache.flink.runtime.rpc.akka; -import akka.actor.ActorRef; -import akka.actor.Status; -import akka.actor.UntypedActor; -import akka.pattern.Patterns; import org.apache.flink.runtime.rpc.MainThreadValidatorUtil; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; import org.apache.flink.runtime.rpc.akka.exceptions.AkkaUnknownMessageException; +import org.apache.flink.runtime.rpc.akka.messages.Processing; +import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.runtime.rpc.messages.CallAsync; -import org.apache.flink.runtime.rpc.messages.ControlMessage; import org.apache.flink.runtime.rpc.messages.LocalRpcInvocation; -import org.apache.flink.runtime.rpc.akka.messages.Processing; import org.apache.flink.runtime.rpc.messages.RpcInvocation; import org.apache.flink.runtime.rpc.messages.RunAsync; - -import org.apache.flink.runtime.rpc.messages.Shutdown; -import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.util.ExceptionUtils; + +import akka.actor.ActorRef; +import akka.actor.Status; +import akka.actor.UntypedActor; +import akka.pattern.Patterns; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.concurrent.duration.FiniteDuration; -import scala.concurrent.impl.Promise; - import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; @@ -50,6 +45,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import scala.concurrent.duration.FiniteDuration; +import scala.concurrent.impl.Promise; + import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -73,38 +71,47 @@ class AkkaRpcActor extends UntypedActor { protected final Logger log = LoggerFactory.getLogger(getClass()); - /** the endpoint to invoke the methods on */ + /** the endpoint to invoke the methods on. */ protected final T rpcEndpoint; - /** the helper that tracks whether calls come from the main thread */ + /** the helper that tracks whether calls come from the main thread. */ private final MainThreadValidatorUtil mainThreadValidator; private final CompletableFuture terminationFuture; - /** Throwable which might have been thrown by the postStop method */ - private Throwable shutdownThrowable; - AkkaRpcActor(final T rpcEndpoint, final CompletableFuture terminationFuture) { this.rpcEndpoint = checkNotNull(rpcEndpoint, "rpc endpoint"); this.mainThreadValidator = new MainThreadValidatorUtil(rpcEndpoint); this.terminationFuture = checkNotNull(terminationFuture); - - this.shutdownThrowable = null; } @Override public void postStop() throws Exception { - super.postStop(); + mainThreadValidator.enterMainThread(); - // IMPORTANT: This only works if we don't use a restarting supervisor strategy. Otherwise - // we would complete the future and let the actor system restart the actor with a completed - // future. - // Complete the termination future so that others know that we've stopped. + try { + Throwable shutdownThrowable = null; - if (shutdownThrowable != null) { - terminationFuture.completeExceptionally(shutdownThrowable); - } else { - terminationFuture.complete(null); + try { + rpcEndpoint.postStop(); + } catch (Throwable throwable) { + shutdownThrowable = throwable; + } + + super.postStop(); + + // IMPORTANT: This only works if we don't use a restarting supervisor strategy. Otherwise + // we would complete the future and let the actor system restart the actor with a completed + // future. + // Complete the termination future so that others know that we've stopped. + + if (shutdownThrowable != null) { + terminationFuture.completeExceptionally(shutdownThrowable); + } else { + terminationFuture.complete(null); + } + } finally { + mainThreadValidator.exitMainThread(); } } @@ -119,11 +126,7 @@ public void onReceive(final Object message) { mainThreadValidator.enterMainThread(); try { - if (msg instanceof ControlMessage) { - handleControlMessage(((ControlMessage) msg)); - } else { - handleMessage(msg); - } + handleMessage(msg); } finally { mainThreadValidator.exitMainThread(); } @@ -139,20 +142,6 @@ public void onReceive(final Object message) { } } - private void handleControlMessage(ControlMessage controlMessage) { - if (controlMessage instanceof Shutdown) { - triggerShutdown(); - } else { - log.warn( - "Received control message of unknown type {} with value {}. Dropping this control message!", - controlMessage.getClass().getName(), - controlMessage); - - sendErrorIfSender(new AkkaUnknownMessageException("Received unknown control message " + controlMessage + - " of type " + controlMessage.getClass().getSimpleName() + '.')); - } - } - protected void handleMessage(Object message) { if (message instanceof RunAsync) { handleRunAsync((RunAsync) message); @@ -186,7 +175,7 @@ private void handleRpcInvocation(RpcInvocation rpcInvocation) { Class[] parameterTypes = rpcInvocation.getParameterTypes(); rpcMethod = lookupRpcMethod(methodName, parameterTypes); - } catch(ClassNotFoundException e) { + } catch (ClassNotFoundException e) { log.error("Could not load method arguments.", e); RpcConnectionException rpcException = new RpcConnectionException("Could not load method arguments.", e); @@ -294,7 +283,7 @@ private void handleRunAsync(RunAsync runAsync) { runAsync.getClass().getName()); } else { - final long timeToRun = runAsync.getTimeNanos(); + final long timeToRun = runAsync.getTimeNanos(); final long delayNanos; if (timeToRun == 0 || (delayNanos = timeToRun - System.nanoTime()) <= 0) { @@ -307,7 +296,7 @@ private void handleRunAsync(RunAsync runAsync) { } } else { - // schedule for later. send a new message after the delay, which will then be immediately executed + // schedule for later. send a new message after the delay, which will then be immediately executed FiniteDuration delay = new FiniteDuration(delayNanos, TimeUnit.NANOSECONDS); RunAsync message = new RunAsync(runAsync.getRunnable(), timeToRun); @@ -317,17 +306,6 @@ private void handleRunAsync(RunAsync runAsync) { } } - private void triggerShutdown() { - try { - rpcEndpoint.postStop(); - } catch (Throwable throwable) { - shutdownThrowable = throwable; - } - - // now stop the actor which will stop processing of any further messages - getContext().system().stop(getSelf()); - } - /** * Look up the rpc method on the given {@link RpcEndpoint} instance. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java index 536a7892659ec..07b334d8d604e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java @@ -42,7 +42,6 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.RpcServer; import org.apache.flink.runtime.rpc.RpcUtils; -import org.apache.flink.runtime.rpc.messages.Shutdown; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -297,7 +296,7 @@ public void stopServer(RpcServer selfGateway) { if (fromThisService) { ActorRef selfActorRef = akkaClient.getRpcEndpoint(); LOG.info("Trigger shut down of RPC endpoint {}.", selfActorRef.path()); - selfActorRef.tell(Shutdown.getInstance(), ActorRef.noSender()); + actorSystem.stop(selfActorRef); } else { LOG.debug("RPC endpoint {} already stopped or from different RPC service"); } @@ -314,11 +313,14 @@ public void stopService() { } stopped = true; + actorSystem.shutdown(); actors.clear(); } actorSystem.awaitTermination(); + + LOG.info("Stopped Akka RPC service."); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/ControlMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/ControlMessage.java deleted file mode 100644 index c16bdd75a1029..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/ControlMessage.java +++ /dev/null @@ -1,26 +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.rpc.messages; - -/** - * Base interface for control messages which are treated separately by the RPC server - * implementation. - */ -public interface ControlMessage { -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/Shutdown.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/Shutdown.java deleted file mode 100644 index 50b076caf6210..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/messages/Shutdown.java +++ /dev/null @@ -1,36 +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.rpc.messages; - -import org.apache.flink.runtime.rpc.akka.AkkaRpcService; - -/** - * Shut down message used to trigger the shut down of an AkkaRpcActor. This - * message is only intended for internal use by the {@link AkkaRpcService}. - */ -public final class Shutdown implements ControlMessage { - - private static Shutdown instance = new Shutdown(); - - public static Shutdown getInstance() { - return instance; - } - - private 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 884668624f00b..da76115006027 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 @@ -31,6 +31,7 @@ import org.apache.flink.runtime.jobmanager.OnCompletionActions; import org.apache.flink.runtime.jobmanager.SubmittedJobGraphStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerServices; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.MetricRegistry; @@ -216,8 +217,8 @@ protected JobManagerRunner createJobManagerRunner( Configuration configuration, RpcService rpcService, HighAvailabilityServices highAvailabilityServices, - BlobServer blobServer, HeartbeatServices heartbeatServices, + JobManagerServices jobManagerServices, MetricRegistry metricRegistry, OnCompletionActions onCompleteActions, FatalErrorHandler fatalErrorHandler) throws Exception { From ba03b78c7703fb372e955f759aa6b70b5f444de9 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 24 Aug 2017 17:26:22 +0200 Subject: [PATCH 090/129] [FLINK-7506] Fence JobMaster This commit lets the JobMaster extend from FencedRpcEndpoint. This enables automatic fencing of all messages. Moreover, this PR introduces the JobMasterId which is the new leader id/fencing token replacing the UUID. This improves type safety when passing multiple fencing tokens around. This closes #4583. --- .../MesosResourceManagerTest.java | 9 +- .../flink/runtime/instance/SlotPool.java | 20 ++- .../runtime/jobmaster/JobManagerRunner.java | 35 ++--- .../flink/runtime/jobmaster/JobMaster.java | 145 +++++------------- .../runtime/jobmaster/JobMasterGateway.java | 20 +-- .../flink/runtime/jobmaster/JobMasterId.java | 60 ++++++++ .../jobmaster/RpcTaskManagerGateway.java | 9 +- .../resourcemanager/JobLeaderIdActions.java | 5 +- .../resourcemanager/JobLeaderIdService.java | 20 +-- .../resourcemanager/ResourceManager.java | 67 ++++---- .../ResourceManagerGateway.java | 10 +- .../registration/JobManagerRegistration.java | 11 +- .../exceptions/LeaderSessionIDException.java | 61 -------- .../taskexecutor/JobLeaderListener.java | 10 +- .../taskexecutor/JobLeaderService.java | 79 +++++----- .../taskexecutor/JobManagerConnection.java | 12 +- .../runtime/taskexecutor/TaskExecutor.java | 75 ++++----- .../taskexecutor/TaskExecutorGateway.java | 6 +- .../rpc/RpcInputSplitProvider.java | 11 +- .../rpc/RpcPartitionStateChecker.java | 7 +- .../RpcResultPartitionConsumableNotifier.java | 7 +- .../clusterframework/ResourceManagerTest.java | 9 +- .../runtime/instance/SlotPoolRpcTest.java | 7 +- .../flink/runtime/instance/SlotPoolTest.java | 16 +- .../jobmaster/JobManagerRunnerMockTest.java | 32 ++-- .../runtime/jobmaster/JobMasterTest.java | 18 +-- .../JobLeaderIdServiceTest.java | 15 +- .../ResourceManagerJobMasterTest.java | 22 +-- .../taskexecutor/TaskExecutorITCase.java | 15 +- .../taskexecutor/TaskExecutorTest.java | 40 ++--- 30 files changed, 357 insertions(+), 496 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterId.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java 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 02b043e7e8758..dbd0746f3da77 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 @@ -47,6 +47,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; @@ -385,7 +386,7 @@ class MockJobMaster { public final ResourceID resourceID; public final String address; public final JobMasterGateway gateway; - public final UUID leaderSessionID; + public final JobMasterId jobMasterId; public final TestingLeaderRetrievalService leaderRetrievalService; MockJobMaster(JobID jobID) { @@ -393,8 +394,8 @@ class MockJobMaster { this.resourceID = new ResourceID(jobID.toString()); this.address = "/" + jobID; this.gateway = mock(JobMasterGateway.class); - this.leaderSessionID = UUID.randomUUID(); - this.leaderRetrievalService = new TestingLeaderRetrievalService(this.address, this.leaderSessionID); + this.jobMasterId = JobMasterId.generate(); + this.leaderRetrievalService = new TestingLeaderRetrievalService(this.address, this.jobMasterId.toUUID()); } } @@ -442,7 +443,7 @@ public void startResourceManager() throws Exception { */ public void registerJobMaster(MockJobMaster jobMaster) throws Exception { CompletableFuture registration = resourceManager.registerJobManager( - jobMaster.leaderSessionID, jobMaster.resourceID, jobMaster.address, jobMaster.jobID, timeout); + jobMaster.jobMasterId, jobMaster.resourceID, jobMaster.address, jobMaster.jobID, timeout); assertTrue(registration.get() instanceof JobMasterRegistrationSuccess); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java index 326e3a2e68d50..6397043562c8a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotPool.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; import org.apache.flink.runtime.jobmanager.slots.SlotAndLocality; import org.apache.flink.runtime.jobmanager.slots.SlotOwner; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.SlotRequest; @@ -53,7 +54,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; @@ -118,8 +118,8 @@ public class SlotPool extends RpcEndpoint implements SlotPoolGateway { private final Clock clock; - /** the leader id of job manager */ - private UUID jobManagerLeaderId; + /** the fencing token of the job manager */ + private JobMasterId jobMasterId; /** The gateway to communicate with resource manager */ private ResourceManagerGateway resourceManagerGateway; @@ -155,6 +155,10 @@ public SlotPool( this.waitingForResourceManager = new HashMap<>(); this.providerAndOwner = new ProviderAndOwner(getSelfGateway(SlotPoolGateway.class), slotRequestTimeout); + + this.jobMasterId = null; + this.resourceManagerGateway = null; + this.jobManagerAddress = null; } // ------------------------------------------------------------------------ @@ -169,11 +173,11 @@ public void start() { /** * Start the slot pool to accept RPC calls. * - * @param newJobManagerLeaderId The necessary leader id for running the job. + * @param jobMasterId The necessary leader id for running the job. * @param newJobManagerAddress for the slot requests which are sent to the resource manager */ - public void start(UUID newJobManagerLeaderId, String newJobManagerAddress) throws Exception { - this.jobManagerLeaderId = checkNotNull(newJobManagerLeaderId); + public void start(JobMasterId jobMasterId, String newJobManagerAddress) throws Exception { + this.jobMasterId = checkNotNull(jobMasterId); this.jobManagerAddress = checkNotNull(newJobManagerAddress); // TODO - start should not throw an exception @@ -195,7 +199,7 @@ public void suspend() { stop(); // do not accept any requests - jobManagerLeaderId = null; + jobMasterId = null; resourceManagerGateway = null; // Clear (but not release!) the available slots. The TaskManagers should re-register them @@ -313,7 +317,7 @@ private void requestSlotFromResourceManager( pendingRequests.put(allocationID, new PendingRequest(allocationID, future, resources)); CompletableFuture rmResponse = resourceManagerGateway.requestSlot( - jobManagerLeaderId, + jobMasterId, new SlotRequest(jobId, allocationID, resources, jobManagerAddress), resourceManagerRequestsTimeout); 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 b5b4b826144c2..6f5a082bf40cd 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 @@ -61,22 +61,22 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F // ------------------------------------------------------------------------ - /** Lock to ensure that this runner can deal with leader election event and job completion notifies simultaneously */ + /** Lock to ensure that this runner can deal with leader election event and job completion notifies simultaneously. */ private final Object lock = new Object(); - /** The job graph needs to run */ + /** The job graph needs to run. */ private final JobGraph jobGraph; - /** The listener to notify once the job completes - either successfully or unsuccessfully */ + /** The listener to notify once the job completes - either successfully or unsuccessfully. */ private final OnCompletionActions toNotifyOnComplete; - /** The handler to call in case of fatal (unrecoverable) errors */ + /** The handler to call in case of fatal (unrecoverable) errors. */ private final FatalErrorHandler errorHandler; - /** Used to check whether a job needs to be run */ + /** Used to check whether a job needs to be run. */ private final RunningJobsRegistry runningJobsRegistry; - /** Leader election for this job */ + /** Leader election for this job. */ private final LeaderElectionService leaderElectionService; private final JobManagerServices jobManagerServices; @@ -87,19 +87,18 @@ public class JobManagerRunner implements LeaderContender, OnCompletionActions, F private final Time timeout; - /** flag marking the runner as shut down */ + /** flag marking the runner as shut down. */ private volatile boolean shutdown; // ------------------------------------------------------------------------ /** - * - *

    Exceptions that occur while creating the JobManager or JobManagerRunner are directly + * Exceptions that occur while creating the JobManager or JobManagerRunner are directly * thrown and not reported to the given {@code FatalErrorHandler}. - * + * *

    This JobManagerRunner assumes that it owns the given {@code JobManagerServices}. * It will shut them down on error and on calls to {@link #shutdown()}. - * + * * @throws Exception Thrown if the runner cannot be set up, because either one of the * required services could not be started, ot the Job could not be initialized. */ @@ -231,7 +230,7 @@ private CompletableFuture shutdownInternally() { //---------------------------------------------------------------------------------------------- /** - * Job completion notification triggered by JobManager + * Job completion notification triggered by JobManager. */ @Override public void jobFinished(JobExecutionResult result) { @@ -247,7 +246,7 @@ public void jobFinished(JobExecutionResult result) { } /** - * Job completion notification triggered by JobManager + * Job completion notification triggered by JobManager. */ @Override public void jobFailed(Throwable cause) { @@ -263,7 +262,7 @@ public void jobFailed(Throwable cause) { } /** - * Job completion notification triggered by self + * Job completion notification triggered by self. */ @Override public void jobFinishedByOther() { @@ -278,7 +277,7 @@ public void jobFinishedByOther() { } /** - * Job completion notification triggered by JobManager or self + * Job completion notification triggered by JobManager or self. */ @Override public void onFatalError(Throwable exception) { @@ -305,7 +304,7 @@ public void onFatalError(Throwable exception) { /** * Marks this runner's job as not running. Other JobManager will not recover the job * after this call. - * + * *

    This method never throws an exception. */ private void unregisterJobFromHighAvailability() { @@ -359,14 +358,14 @@ public void grantLeadership(final UUID leaderSessionID) { // This will eventually be noticed, but can not be ruled out from the beginning. if (leaderElectionService.hasLeadership()) { try { - // Now set the running status is after getting leader ship and + // Now set the running status is after getting leader ship and // set finished status after job in terminated status. // So if finding the job is running, it means someone has already run the job, need recover. if (schedulingStatus == JobSchedulingStatus.PENDING) { runningJobsRegistry.setJobRunning(jobGraph.getJobID()); } - CompletableFuture startingFuture = jobManager.start(leaderSessionID, timeout); + CompletableFuture startingFuture = jobManager.start(new JobMasterId(leaderSessionID), timeout); startingFuture.whenCompleteAsync( (Acknowledge ack, Throwable 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 a05242aafb42f..80d6e4f496b41 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 @@ -53,7 +53,6 @@ import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.HeartbeatTarget; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.LeaderIdMismatchException; import org.apache.flink.runtime.instance.Slot; import org.apache.flink.runtime.instance.SlotPool; import org.apache.flink.runtime.instance.SlotPoolGateway; @@ -83,7 +82,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcEndpoint; +import org.apache.flink.runtime.rpc.FencedRpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils; import org.apache.flink.runtime.state.KeyGroupRange; @@ -125,7 +124,7 @@ * given task *

*/ -public class JobMaster extends RpcEndpoint implements JobMasterGateway { +public class JobMaster extends FencedRpcEndpoint implements JobMasterGateway { /** Default names for Flink's distributed components */ public static final String JOB_MANAGER_NAME = "jobmanager"; @@ -182,8 +181,6 @@ public class JobMaster extends RpcEndpoint implements JobMasterGateway { private final SlotPoolGateway slotPoolGateway; - private volatile UUID leaderSessionID; - // --------- ResourceManager -------- /** Leader retriever service used to locate ResourceManager's address */ @@ -215,7 +212,7 @@ public JobMaster( FatalErrorHandler errorHandler, ClassLoader userCodeLoader) throws Exception { - super(rpcService, AkkaRpcServiceUtils.createRandomName(JobMaster.JOB_MANAGER_NAME)); + super(rpcService, AkkaRpcServiceUtils.createRandomName(JobMaster.JOB_MANAGER_NAME), JobMasterId.INITIAL_JOB_MASTER_ID); selfGateway = getSelfGateway(JobMasterGateway.class); @@ -308,15 +305,15 @@ public void start() { /** * Start the rpc service and begin to run the job. * - * @param leaderSessionID The necessary leader id for running the job. + * @param newJobMasterId The necessary fencing token to run the job * @param timeout for the operation * @return Future acknowledge if the job could be started. Otherwise the future contains an exception */ - public CompletableFuture start(final UUID leaderSessionID, final Time timeout) throws Exception { + public CompletableFuture start(final JobMasterId newJobMasterId, final Time timeout) throws Exception { // make sure we receive RPC and async calls super.start(); - return callAsync(() -> startJobExecution(leaderSessionID), timeout); + return callAsyncWithoutFencing(() -> startJobExecution(newJobMasterId), timeout); } /** @@ -324,7 +321,7 @@ public CompletableFuture start(final UUID leaderSessionID, final Ti * will be disposed. * *

Mostly job is suspended because of the leadership has been revoked, one can be restart this job by - * calling the {@link #start(UUID, Time)} method once we take the leadership back again. + * calling the {@link #start(JobMasterId, Time)} method once we take the leadership back again. * *

This method is executed asynchronously * @@ -333,7 +330,7 @@ public CompletableFuture start(final UUID leaderSessionID, final Ti * @return Future acknowledge indicating that the job has been suspended. Otherwise the future contains an exception */ public CompletableFuture suspend(final Throwable cause, final Time timeout) { - CompletableFuture suspendFuture = callAsync(() -> suspendExecution(cause), timeout); + CompletableFuture suspendFuture = callAsyncWithoutFencing(() -> suspendExecution(cause), timeout); stop(); @@ -366,17 +363,10 @@ public void postStop() throws Exception { */ @Override public CompletableFuture updateTaskExecutionState( - final UUID leaderSessionID, final TaskExecutionState taskExecutionState) { checkNotNull(taskExecutionState, "taskExecutionState"); - try { - validateLeaderSessionId(leaderSessionID); - } catch (LeaderIdMismatchException e) { - return FutureUtils.completedExceptionally(e); - } - if (executionGraph.updateState(taskExecutionState)) { return CompletableFuture.completedFuture(Acknowledge.get()); } else { @@ -388,16 +378,9 @@ public CompletableFuture updateTaskExecutionState( @Override public CompletableFuture requestNextInputSplit( - final UUID leaderSessionID, final JobVertexID vertexID, final ExecutionAttemptID executionAttempt) { - try { - validateLeaderSessionId(leaderSessionID); - } catch (LeaderIdMismatchException e) { - return FutureUtils.completedExceptionally(e); - } - final Execution execution = executionGraph.getRegisteredExecutions().get(executionAttempt); if (execution == null) { // can happen when JobManager had already unregistered this execution upon on task failure, @@ -444,16 +427,9 @@ public CompletableFuture requestNextInputSplit( @Override public CompletableFuture requestPartitionState( - final UUID leaderSessionID, final IntermediateDataSetID intermediateResultId, final ResultPartitionID resultPartitionId) { - try { - validateLeaderSessionId(leaderSessionID); - } catch (LeaderIdMismatchException e) { - return FutureUtils.completedExceptionally(e); - } - final Execution execution = executionGraph.getRegisteredExecutions().get(resultPartitionId.getProducerId()); if (execution != null) { return CompletableFuture.completedFuture(execution.getState()); @@ -483,12 +459,9 @@ public CompletableFuture requestPartitionState( @Override public CompletableFuture scheduleOrUpdateConsumers( - final UUID leaderSessionID, final ResultPartitionID partitionID, final Time timeout) { try { - validateLeaderSessionId(leaderSessionID); - executionGraph.scheduleOrUpdateConsumers(partitionID); return CompletableFuture.completedFuture(Acknowledge.get()); } catch (Exception e) { @@ -636,15 +609,8 @@ public CompletableFuture requestClassloadingProps() { public CompletableFuture> offerSlots( final ResourceID taskManagerId, final Iterable slots, - final UUID leaderId, final Time timeout) { - try { - validateLeaderSessionId(leaderId); - } catch (LeaderIdMismatchException e) { - return FutureUtils.completedExceptionally(e); - } - Tuple2 taskManager = registeredTaskManagers.get(taskManagerId); if (taskManager == null) { @@ -657,7 +623,7 @@ public CompletableFuture> offerSlots( final ArrayList> slotsAndOffers = new ArrayList<>(); - final RpcTaskManagerGateway rpcTaskManagerGateway = new RpcTaskManagerGateway(taskExecutorGateway, leaderId); + final RpcTaskManagerGateway rpcTaskManagerGateway = new RpcTaskManagerGateway(taskExecutorGateway, getFencingToken()); for (SlotOffer slotOffer : slots) { final AllocatedSlot slot = new AllocatedSlot( @@ -678,15 +644,8 @@ public CompletableFuture> offerSlots( public void failSlot( final ResourceID taskManagerId, final AllocationID allocationId, - final UUID leaderId, final Exception cause) { - try { - validateLeaderSessionId(leaderSessionID); - } catch (LeaderIdMismatchException e) { - log.warn("Cannot fail slot " + allocationId + '.', e); - } - if (registeredTaskManagers.containsKey(taskManagerId)) { slotPoolGateway.failAllocation(allocationId, cause); } else { @@ -699,16 +658,7 @@ public void failSlot( public CompletableFuture registerTaskManager( final String taskManagerRpcAddress, final TaskManagerLocation taskManagerLocation, - final UUID leaderId, final Time timeout) { - if (!Objects.equals(leaderSessionID, leaderId)) { - log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " + - "leader session ID {} did not equal the received leader session ID {}.", - taskManagerLocation.getResourceID(), taskManagerRpcAddress, leaderSessionID, leaderId); - return FutureUtils.completedExceptionally( - new Exception("Leader id not match, expected: " + - leaderSessionID + ", actual: " + leaderId)); - } final ResourceID taskManagerId = taskManagerLocation.getResourceID(); @@ -725,13 +675,6 @@ public CompletableFuture registerTaskManager( return new RegistrationResponse.Decline(throwable.getMessage()); } - if (!Objects.equals(leaderSessionID, leaderId)) { - log.warn("Discard registration from TaskExecutor {} at ({}) because the expected " + - "leader session ID {} did not equal the received leader session ID {}.", - taskManagerId, taskManagerRpcAddress, leaderSessionID, leaderId); - return new RegistrationResponse.Decline("Invalid leader session id"); - } - slotPoolGateway.registerTaskManager(taskManagerId); registeredTaskManagers.put(taskManagerId, Tuple2.of(taskManagerLocation, taskExecutorGateway)); @@ -756,16 +699,9 @@ public void requestHeartbeat(ResourceID resourceID, Void payload) { @Override public void disconnectResourceManager( - final UUID jobManagerLeaderId, final ResourceManagerId resourceManagerId, final Exception cause) { - try { - validateLeaderSessionId(jobManagerLeaderId); - } catch (LeaderIdMismatchException e) { - log.warn("Cannot disconnect resource manager " + resourceManagerId + '.', e); - } - if (resourceManagerConnection != null && resourceManagerConnection.getTargetLeaderId().equals(resourceManagerId)) { closeResourceManagerConnection(cause); @@ -788,34 +724,34 @@ public void heartbeatFromResourceManager(final ResourceID resourceID) { //-- job starting and stopping ----------------------------------------------------------------- - private Acknowledge startJobExecution(UUID newLeaderSessionId) throws Exception { - Preconditions.checkNotNull(newLeaderSessionId, "The new leader session id must not be null."); + private Acknowledge startJobExecution(JobMasterId newJobMasterId) throws Exception { + validateRunsInMainThread(); - if (leaderSessionID == null) { - log.info("Start job execution with leader id {}.", newLeaderSessionId); + Preconditions.checkNotNull(newJobMasterId, "The new JobMasterId must not be null."); - leaderSessionID = newLeaderSessionId; - } else if (Objects.equals(leaderSessionID, newLeaderSessionId)) { - log.info("Already started the job execution with leader id {}.", leaderSessionID); + if (Objects.equals(getFencingToken(), newJobMasterId)) { + log.info("Already started the job execution with JobMasterId {}.", newJobMasterId); return Acknowledge.get(); - } else { - log.info("Restarting old job with leader id {}. The new leader id is {}.", leaderSessionID, newLeaderSessionId); + } - // first we have to suspend the current execution - suspendExecution(new FlinkException("Old job with leader id " + leaderSessionID + - " is restarted with a new leader id " + newLeaderSessionId + '.')); + if (!Objects.equals(getFencingToken(), JobMasterId.INITIAL_JOB_MASTER_ID)) { + log.info("Restarting old job with JobMasterId {}. The new JobMasterId is {}.", getFencingToken(), newJobMasterId); - // set new leader id - leaderSessionID = newLeaderSessionId; + // first we have to suspend the current execution + suspendExecution(new FlinkException("Old job with JobMasterId " + getFencingToken() + + " is restarted with a new JobMasterId " + newJobMasterId + '.')); } + // set new leader id + setFencingToken(newJobMasterId); + log.info("Starting execution of job {} ({})", jobGraph.getName(), jobGraph.getJobID()); try { // start the slot pool make sure the slot pool now accepts messages for this leader log.debug("Staring SlotPool component"); - slotPool.start(leaderSessionID, getAddress()); + slotPool.start(getFencingToken(), getAddress()); // job is ready to go, try to establish connection with resource manager // - activate leader retrieval for the resource manager @@ -848,18 +784,20 @@ private Acknowledge startJobExecution(UUID newLeaderSessionId) throws Exception * will be disposed. * *

Mostly job is suspended because of the leadership has been revoked, one can be restart this job by - * calling the {@link #start(UUID, Time)} method once we take the leadership back again. + * calling the {@link #start(JobMasterId, Time)} method once we take the leadership back again. * * @param cause The reason of why this job been suspended. */ private Acknowledge suspendExecution(final Throwable cause) { - if (leaderSessionID == null) { + validateRunsInMainThread(); + + if (getFencingToken() == null) { log.debug("Job has already been suspended or shutdown."); return Acknowledge.get(); } - // not leader any more - should not accept any leader messages any more - leaderSessionID = null; + // not leader anymore --> set the JobMasterId to the initial id + setFencingToken(JobMasterId.INITIAL_JOB_MASTER_ID); try { resourceManagerLeaderRetriever.stop(); @@ -973,7 +911,7 @@ private void notifyOfNewResourceManagerLeader(final String resourceManagerAddres jobGraph.getJobID(), resourceId, getAddress(), - leaderSessionID, + getFencingToken(), resourceManagerAddress, resourceManagerId, executor); @@ -1028,12 +966,6 @@ private void closeResourceManagerConnection(Exception cause) { slotPoolGateway.disconnectResourceManager(); } - private void validateLeaderSessionId(UUID leaderSessionID) throws LeaderIdMismatchException { - if (this.leaderSessionID == null || !this.leaderSessionID.equals(leaderSessionID)) { - throw new LeaderIdMismatchException(this.leaderSessionID, leaderSessionID); - } - } - //---------------------------------------------------------------------------------------------- // Utility classes //---------------------------------------------------------------------------------------------- @@ -1065,7 +997,7 @@ private class ResourceManagerConnection private final String jobManagerRpcAddress; - private final UUID jobManagerLeaderID; + private final JobMasterId jobMasterId; private ResourceID resourceManagerResourceID; @@ -1074,7 +1006,7 @@ private class ResourceManagerConnection final JobID jobID, final ResourceID jobManagerResourceID, final String jobManagerRpcAddress, - final UUID jobManagerLeaderID, + final JobMasterId jobMasterId, final String resourceManagerAddress, final ResourceManagerId resourceManagerId, final Executor executor) @@ -1083,7 +1015,7 @@ private class ResourceManagerConnection this.jobID = checkNotNull(jobID); this.jobManagerResourceID = checkNotNull(jobManagerResourceID); this.jobManagerRpcAddress = checkNotNull(jobManagerRpcAddress); - this.jobManagerLeaderID = checkNotNull(jobManagerLeaderID); + this.jobMasterId = checkNotNull(jobMasterId); } @Override @@ -1099,7 +1031,7 @@ protected CompletableFuture invokeRegistration( Time timeout = Time.milliseconds(timeoutMillis); return gateway.registerJobManager( - jobManagerLeaderID, + jobMasterId, jobManagerResourceID, jobManagerRpcAddress, jobID, @@ -1145,12 +1077,7 @@ public void jobStatusChanges( final Throwable error) { // run in rpc thread to avoid concurrency - runAsync(new Runnable() { - @Override - public void run() { - jobStatusChanged(newJobStatus, timestamp, error); - } - }); + runAsync(() -> jobStatusChanged(newJobStatus, timestamp, error)); } } 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 b39f419bbd446..965d88d272e3c 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 @@ -37,6 +37,7 @@ import org.apache.flink.runtime.query.KvStateServerAddress; import org.apache.flink.runtime.registration.RegistrationResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; +import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.state.internal.InternalKvState; import org.apache.flink.runtime.state.KeyGroupRange; @@ -45,36 +46,31 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import java.util.Collection; -import java.util.UUID; import java.util.concurrent.CompletableFuture; /** * {@link JobMaster} rpc gateway interface */ -public interface JobMasterGateway extends CheckpointCoordinatorGateway { +public interface JobMasterGateway extends CheckpointCoordinatorGateway, FencedRpcGateway { /** * Updates the task execution state for a given task. * - * @param leaderSessionID The leader id of JobManager * @param taskExecutionState New task execution state for a given task * @return Future flag of the task execution state update result */ CompletableFuture updateTaskExecutionState( - final UUID leaderSessionID, final TaskExecutionState taskExecutionState); /** * Requesting next input split for the {@link ExecutionJobVertex}. The next input split is sent back to the sender * as a {@link SerializedInputSplit} message. * - * @param leaderSessionID The leader id of JobManager * @param vertexID The job vertex id * @param executionAttempt The execution attempt id * @return The future of the input split. If there is no further input split, will return an empty object. */ CompletableFuture requestNextInputSplit( - final UUID leaderSessionID, final JobVertexID vertexID, final ExecutionAttemptID executionAttempt); @@ -82,13 +78,11 @@ CompletableFuture requestNextInputSplit( * Requests the current state of the partition. * The state of a partition is currently bound to the state of the producing execution. * - * @param leaderSessionID The leader id of JobManager * @param intermediateResultId The execution attempt ID of the task requesting the partition state. * @param partitionId The partition ID of the partition to request the state of. * @return The future of the partition state */ CompletableFuture requestPartitionState( - final UUID leaderSessionID, final IntermediateDataSetID intermediateResultId, final ResultPartitionID partitionId); @@ -101,13 +95,11 @@ CompletableFuture requestPartitionState( *

* The JobManager then can decide when to schedule the partition consumers of the given session. * - * @param leaderSessionID The leader id of JobManager * @param partitionID The partition which has already produced data * @param timeout before the rpc call fails * @return Future acknowledge of the schedule or update operation */ CompletableFuture scheduleOrUpdateConsumers( - final UUID leaderSessionID, final ResultPartitionID partitionID, @RpcTimeout final Time timeout); @@ -123,12 +115,10 @@ CompletableFuture scheduleOrUpdateConsumers( /** * Disconnects the resource manager from the job manager because of the given cause. * - * @param jobManagerLeaderId identifying the job manager leader id * @param resourceManagerId identifying the resource manager leader id * @param cause of the disconnect */ void disconnectResourceManager( - final UUID jobManagerLeaderId, final ResourceManagerId resourceManagerId, final Exception cause); @@ -174,14 +164,12 @@ void notifyKvStateUnregistered( * * @param taskManagerId identifying the task manager * @param slots to offer to the job manager - * @param leaderId identifying the job leader * @param timeout for the rpc call * @return Future set of accepted slots. */ CompletableFuture> offerSlots( final ResourceID taskManagerId, final Iterable slots, - final UUID leaderId, @RpcTimeout final Time timeout); /** @@ -189,12 +177,10 @@ CompletableFuture> offerSlots( * * @param taskManagerId identifying the task manager * @param allocationId identifying the slot to fail - * @param leaderId identifying the job leader * @param cause of the failing */ void failSlot(final ResourceID taskManagerId, final AllocationID allocationId, - final UUID leaderId, final Exception cause); /** @@ -202,14 +188,12 @@ void failSlot(final ResourceID taskManagerId, * * @param taskManagerRpcAddress the rpc address of the task manager * @param taskManagerLocation location of the task manager - * @param leaderId identifying the job leader * @param timeout for the rpc call * @return Future registration response indicating whether the registration was successful or not */ CompletableFuture registerTaskManager( final String taskManagerRpcAddress, final TaskManagerLocation taskManagerLocation, - final UUID leaderId, @RpcTimeout final Time timeout); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterId.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterId.java new file mode 100644 index 0000000000000..ffd53b31b6864 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterId.java @@ -0,0 +1,60 @@ +/* + * 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.util.AbstractID; + +import java.util.UUID; + +/** + * The {@link JobMaster} fencing token. + */ +public class JobMasterId extends AbstractID { + + private static final long serialVersionUID = -933276753644003754L; + + public static final JobMasterId INITIAL_JOB_MASTER_ID = new JobMasterId(0L, 0L); + + public JobMasterId(byte[] bytes) { + super(bytes); + } + + public JobMasterId(long lowerPart, long upperPart) { + super(lowerPart, upperPart); + } + + public JobMasterId(AbstractID id) { + super(id); + } + + public JobMasterId() { + } + + public JobMasterId(UUID uuid) { + this(uuid.getLeastSignificantBits(), uuid.getMostSignificantBits()); + } + + public UUID toUUID() { + return new UUID(getUpperPart(), getLowerPart()); + } + + public static JobMasterId generate() { + return new JobMasterId(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java index e93c907d36b6b..8967aae9a2f8b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/RpcTaskManagerGateway.java @@ -33,7 +33,6 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.util.Preconditions; -import java.util.UUID; import java.util.concurrent.CompletableFuture; /** @@ -43,11 +42,11 @@ public class RpcTaskManagerGateway implements TaskManagerGateway { private final TaskExecutorGateway taskExecutorGateway; - private final UUID leaderId; + private final JobMasterId jobMasterId; - public RpcTaskManagerGateway(TaskExecutorGateway taskExecutorGateway, UUID leaderId) { + public RpcTaskManagerGateway(TaskExecutorGateway taskExecutorGateway, JobMasterId jobMasterId) { this.taskExecutorGateway = Preconditions.checkNotNull(taskExecutorGateway); - this.leaderId = Preconditions.checkNotNull(leaderId); + this.jobMasterId = Preconditions.checkNotNull(jobMasterId); } @Override @@ -87,7 +86,7 @@ public CompletableFuture requestStackTraceSample( @Override public CompletableFuture submitTask(TaskDeploymentDescriptor tdd, Time timeout) { - return taskExecutorGateway.submitTask(tdd, leaderId, timeout); + return taskExecutorGateway.submitTask(tdd, jobMasterId, timeout); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdActions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdActions.java index 4ca62090a098b..565cd82918739 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdActions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdActions.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.resourcemanager; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobmaster.JobMasterId; import java.util.UUID; @@ -31,9 +32,9 @@ public interface JobLeaderIdActions { * Callback when a monitored job leader lost its leadership. * * @param jobId identifying the job whose leader lost leadership - * @param oldJobLeaderId of the job manager which lost leadership + * @param oldJobMasterId of the job manager which lost leadership */ - void jobLeaderLostLeadership(JobID jobId, UUID oldJobLeaderId); + void jobLeaderLostLeadership(JobID jobId, JobMasterId oldJobMasterId); /** * Notify a job timeout. The job is identified by the given JobID. In order to check diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java index aaa72d9fc78ad..da0a7fd5b5b73 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdService.java @@ -22,14 +22,17 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +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.util.ExceptionUtils; import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; + import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -49,17 +52,17 @@ public class JobLeaderIdService { private static final Logger LOG = LoggerFactory.getLogger(JobLeaderIdService.class); - /** High availability services to use by this service */ + /** High availability services to use by this service. */ private final HighAvailabilityServices highAvailabilityServices; private final ScheduledExecutor scheduledExecutor; private final Time jobTimeout; - /** Map of currently monitored jobs */ + /** Map of currently monitored jobs. */ private final Map jobLeaderIdListeners; - /** Actions to call when the job leader changes */ + /** Actions to call when the job leader changes. */ private JobLeaderIdActions jobLeaderIdActions; public JobLeaderIdService( @@ -178,14 +181,14 @@ public boolean containsJob(JobID jobId) { return jobLeaderIdListeners.containsKey(jobId); } - public CompletableFuture getLeaderId(JobID jobId) throws Exception { + public CompletableFuture getLeaderId(JobID jobId) throws Exception { if (!jobLeaderIdListeners.containsKey(jobId)) { addJob(jobId); } JobLeaderIdListener listener = jobLeaderIdListeners.get(jobId); - return listener.getLeaderIdFuture(); + return listener.getLeaderIdFuture().thenApply((UUID id) -> id != null ? new JobMasterId(id) : null); } public boolean isValidTimeout(JobID jobId, UUID timeoutId) { @@ -216,15 +219,14 @@ private final class JobLeaderIdListener implements LeaderRetrievalListener { private volatile CompletableFuture leaderIdFuture; private volatile boolean running = true; - /** Null if no timeout has been scheduled; otherwise non null */ + /** Null if no timeout has been scheduled; otherwise non null. */ @Nullable private volatile ScheduledFuture timeoutFuture; - /** Null if no timeout has been scheduled; otherwise non null */ + /** Null if no timeout has been scheduled; otherwise non null. */ @Nullable private volatile UUID timeoutId; - private JobLeaderIdListener( JobID jobId, JobLeaderIdActions listenerJobLeaderIdActions, @@ -279,7 +281,7 @@ public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionId) { if (previousJobLeaderId != null && !previousJobLeaderId.equals(leaderSessionId)) { // we had a previous job leader, so notify about his lost leadership - listenerJobLeaderIdActions.jobLeaderLostLeadership(jobId, previousJobLeaderId); + listenerJobLeaderIdActions.jobLeaderLostLeadership(jobId, new JobMasterId(previousJobLeaderId)); if (null == leaderSessionId) { // No current leader active ==> Set a timeout for the job 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 659b3d4073d33..87cf7d10fbcaa 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 @@ -36,6 +36,7 @@ import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -51,7 +52,6 @@ import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.FencedRpcEndpoint; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.exceptions.LeaderSessionIDException; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; @@ -75,8 +75,8 @@ * *

It offers the following methods as part of its rpc interface to interact with him remotely: *

    - *
  • {@link #registerJobManager(UUID, ResourceID, String, JobID, Time)} registers a {@link JobMaster} at the resource manager
  • - *
  • {@link #requestSlot(UUID, SlotRequest, Time)} requests a slot from the resource manager
  • + *
  • {@link #registerJobManager(JobMasterId, ResourceID, String, JobID, Time)} registers a {@link JobMaster} at the resource manager
  • + *
  • {@link #requestSlot(JobMasterId, SlotRequest, Time)} requests a slot from the resource manager
  • *
*/ public abstract class ResourceManager @@ -240,13 +240,13 @@ public void postStop() throws Exception { @Override public CompletableFuture registerJobManager( - final UUID jobManagerLeaderId, + final JobMasterId jobMasterId, final ResourceID jobManagerResourceId, final String jobManagerAddress, final JobID jobId, final Time timeout) { - checkNotNull(jobManagerLeaderId); + checkNotNull(jobMasterId); checkNotNull(jobManagerResourceId); checkNotNull(jobManagerAddress); checkNotNull(jobId); @@ -265,12 +265,12 @@ public CompletableFuture registerJobManager( } } - log.info("Registering job manager {}@{} for job {}.", jobManagerLeaderId, jobManagerAddress, jobId); + log.info("Registering job manager {}@{} for job {}.", jobMasterId, jobManagerAddress, jobId); - CompletableFuture jobLeaderIdFuture; + CompletableFuture jobMasterIdFuture; try { - jobLeaderIdFuture = jobLeaderIdService.getLeaderId(jobId); + jobMasterIdFuture = jobLeaderIdService.getLeaderId(jobId); } catch (Exception e) { // we cannot check the job leader id so let's fail // TODO: Maybe it's also ok to skip this check in case that we cannot check the leader id @@ -283,21 +283,20 @@ public CompletableFuture registerJobManager( return FutureUtils.completedExceptionally(exception); } - CompletableFuture jobMasterGatewayFuture = getRpcService().connect(jobManagerAddress, JobMasterGateway.class); + CompletableFuture jobMasterGatewayFuture = getRpcService().connect(jobManagerAddress, jobMasterId, JobMasterGateway.class); CompletableFuture registrationResponseFuture = jobMasterGatewayFuture.thenCombineAsync( - jobLeaderIdFuture, - (JobMasterGateway jobMasterGateway, UUID jobLeaderId) -> { - if (Objects.equals(jobLeaderId, jobManagerLeaderId)) { + jobMasterIdFuture, + (JobMasterGateway jobMasterGateway, JobMasterId currentJobMasterId) -> { + if (Objects.equals(currentJobMasterId, jobMasterId)) { return registerJobMasterInternal( jobMasterGateway, - jobLeaderId, jobId, jobManagerAddress, jobManagerResourceId); } else { - log.debug("The job manager leader id {} did not match the job " + - "leader id {}.", jobManagerLeaderId, jobLeaderId); + log.debug("The current JobMaster leader id {} did not match the received " + + "JobMaster id {}.", jobMasterId, currentJobMasterId); return new RegistrationResponse.Decline("Job manager leader id did not match."); } }, @@ -308,9 +307,9 @@ public CompletableFuture registerJobManager( (RegistrationResponse registrationResponse, Throwable throwable) -> { if (throwable != null) { if (log.isDebugEnabled()) { - log.debug("Registration of job manager {}@{} failed.", jobManagerLeaderId, jobManagerAddress, throwable); + log.debug("Registration of job manager {}@{} failed.", jobMasterId, jobManagerAddress, throwable); } else { - log.info("Registration of job manager {}@{} failed.", jobManagerLeaderId, jobManagerAddress); + log.info("Registration of job manager {}@{} failed.", jobMasterId, jobManagerAddress); } return new RegistrationResponse.Decline(throwable.getMessage()); @@ -367,7 +366,7 @@ public void disconnectJobManager(final JobID jobId, final Exception cause) { @Override public CompletableFuture requestSlot( - UUID jobMasterLeaderID, + JobMasterId jobMasterId, SlotRequest slotRequest, final Time timeout) { @@ -375,7 +374,7 @@ public CompletableFuture requestSlot( JobManagerRegistration jobManagerRegistration = jobManagerRegistrations.get(jobId); if (null != jobManagerRegistration) { - if (Objects.equals(jobMasterLeaderID, jobManagerRegistration.getLeaderID())) { + if (Objects.equals(jobMasterId, jobManagerRegistration.getJobMasterId())) { log.info("Request slot with profile {} for job {} with allocation id {}.", slotRequest.getResourceProfile(), slotRequest.getJobId(), @@ -389,7 +388,8 @@ public CompletableFuture requestSlot( return CompletableFuture.completedFuture(Acknowledge.get()); } else { - return FutureUtils.completedExceptionally(new LeaderSessionIDException(jobMasterLeaderID, jobManagerRegistration.getLeaderID())); + return FutureUtils.completedExceptionally(new ResourceManagerException("The job leader's id " + + jobManagerRegistration.getJobMasterId() + " does not match the received id " + jobMasterId + '.')); } } else { @@ -488,7 +488,6 @@ public CompletableFuture getNumberOfRegisteredTaskManagers() { * Registers a new JobMaster. * * @param jobMasterGateway to communicate with the registering JobMaster - * @param jobLeaderId leader id of the JobMaster * @param jobId of the job for which the JobMaster is responsible * @param jobManagerAddress address of the JobMaster * @param jobManagerResourceId ResourceID of the JobMaster @@ -496,16 +495,15 @@ public CompletableFuture getNumberOfRegisteredTaskManagers() { */ private RegistrationResponse registerJobMasterInternal( final JobMasterGateway jobMasterGateway, - UUID jobLeaderId, JobID jobId, String jobManagerAddress, ResourceID jobManagerResourceId) { if (jobManagerRegistrations.containsKey(jobId)) { JobManagerRegistration oldJobManagerRegistration = jobManagerRegistrations.get(jobId); - if (oldJobManagerRegistration.getLeaderID().equals(jobLeaderId)) { + if (Objects.equals(oldJobManagerRegistration.getJobMasterId(), jobMasterGateway.getFencingToken())) { // same registration - log.debug("Job manager {}@{} was already registered.", jobLeaderId, jobManagerAddress); + log.debug("Job manager {}@{} was already registered.", jobMasterGateway.getFencingToken(), jobManagerAddress); } else { // tell old job manager that he is no longer the job leader disconnectJobManager( @@ -515,7 +513,6 @@ private RegistrationResponse registerJobMasterInternal( JobManagerRegistration jobManagerRegistration = new JobManagerRegistration( jobId, jobManagerResourceId, - jobLeaderId, jobMasterGateway); jobManagerRegistrations.put(jobId, jobManagerRegistration); jmResourceIdRegistrations.put(jobManagerResourceId, jobManagerRegistration); @@ -525,13 +522,12 @@ private RegistrationResponse registerJobMasterInternal( JobManagerRegistration jobManagerRegistration = new JobManagerRegistration( jobId, jobManagerResourceId, - jobLeaderId, jobMasterGateway); jobManagerRegistrations.put(jobId, jobManagerRegistration); jmResourceIdRegistrations.put(jobManagerResourceId, jobManagerRegistration); } - log.info("Registered job manager {}@{} for job {}.", jobLeaderId, jobManagerAddress, jobId); + log.info("Registered job manager {}@{} for job {}.", jobMasterGateway.getFencingToken(), jobManagerAddress, jobId); jobManagerHeartbeatManager.monitorTarget(jobManagerResourceId, new HeartbeatTarget() { @Override @@ -633,10 +629,10 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) { if (jobManagerRegistration != null) { final ResourceID jobManagerResourceId = jobManagerRegistration.getJobManagerResourceID(); final JobMasterGateway jobMasterGateway = jobManagerRegistration.getJobManagerGateway(); - final UUID jobManagerLeaderId = jobManagerRegistration.getLeaderID(); + final JobMasterId jobMasterId = jobManagerRegistration.getJobMasterId(); log.info("Disconnect job manager {}@{} for job {} from the resource manager.", - jobManagerLeaderId, + jobMasterId, jobMasterGateway.getAddress(), jobId); @@ -645,7 +641,7 @@ protected void closeJobManagerConnection(JobID jobId, Exception cause) { jmResourceIdRegistrations.remove(jobManagerResourceId); // tell the job manager about the disconnect - jobMasterGateway.disconnectResourceManager(jobManagerLeaderId, getFencingToken(), cause); + jobMasterGateway.disconnectResourceManager(getFencingToken(), cause); } else { log.debug("There was no registered job manager for job {}.", jobId); } @@ -687,17 +683,17 @@ protected void removeJob(JobID jobId) { } } - protected void jobLeaderLostLeadership(JobID jobId, UUID oldJobLeaderId) { + protected void jobLeaderLostLeadership(JobID jobId, JobMasterId oldJobMasterId) { if (jobManagerRegistrations.containsKey(jobId)) { JobManagerRegistration jobManagerRegistration = jobManagerRegistrations.get(jobId); - if (Objects.equals(jobManagerRegistration.getLeaderID(), oldJobLeaderId)) { + if (Objects.equals(jobManagerRegistration.getJobMasterId(), oldJobMasterId)) { disconnectJobManager(jobId, new Exception("Job leader lost leadership.")); } else { log.debug("Discarding job leader lost leadership, because a new job leader was found for job {}. ", jobId); } } else { - log.debug("Discard job leader lost leadership for outdated leader {} for job {}.", oldJobLeaderId, jobId); + log.debug("Discard job leader lost leadership for outdated leader {} for job {}.", oldJobMasterId, jobId); } } @@ -824,6 +820,7 @@ public void handleError(final Exception exception) { /** * Allocates a resource using the resource profile. + * * @param resourceProfile The resource description */ @VisibleForTesting @@ -891,11 +888,11 @@ public void notifyAllocationFailure(JobID jobId, AllocationID allocationId, Exce private class JobLeaderIdActionsImpl implements JobLeaderIdActions { @Override - public void jobLeaderLostLeadership(final JobID jobId, final UUID oldJobLeaderId) { + public void jobLeaderLostLeadership(final JobID jobId, final JobMasterId oldJobMasterId) { runAsync(new Runnable() { @Override public void run() { - ResourceManager.this.jobLeaderLostLeadership(jobId, oldJobLeaderId); + ResourceManager.this.jobLeaderLostLeadership(jobId, oldJobMasterId); } }); } 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 ac81048acd781..a957716aeb66e 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 @@ -25,6 +25,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.SlotID; import org.apache.flink.runtime.instance.InstanceID; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.FencedRpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; @@ -33,7 +34,6 @@ import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.TaskExecutor; -import java.util.UUID; import java.util.concurrent.CompletableFuture; /** @@ -44,7 +44,7 @@ public interface ResourceManagerGateway extends FencedRpcGateway registerJobManager( - UUID jobMasterLeaderId, + JobMasterId jobMasterId, ResourceID jobMasterResourceId, String jobMasterAddress, JobID jobId, @@ -61,12 +61,12 @@ CompletableFuture registerJobManager( /** * Requests a slot from the resource manager. * - * @param jobMasterLeaderID leader if of the JobMaster + * @param jobMasterId id of the JobMaster * @param slotRequest The slot to request * @return The confirmation that the slot gets allocated */ CompletableFuture requestSlot( - UUID jobMasterLeaderID, + JobMasterId jobMasterId, SlotRequest slotRequest, @RpcTimeout Time timeout); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobManagerRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobManagerRegistration.java index df3a39fad2856..dca2db674edfe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobManagerRegistration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/registration/JobManagerRegistration.java @@ -21,10 +21,9 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.util.Preconditions; -import java.util.UUID; - /** * Container for JobManager related registration information, such as the leader id or the job id. */ @@ -33,18 +32,14 @@ public class JobManagerRegistration { private final ResourceID jobManagerResourceID; - private final UUID leaderID; - private final JobMasterGateway jobManagerGateway; public JobManagerRegistration( JobID jobID, ResourceID jobManagerResourceID, - UUID leaderID, JobMasterGateway jobManagerGateway) { this.jobID = Preconditions.checkNotNull(jobID); this.jobManagerResourceID = Preconditions.checkNotNull(jobManagerResourceID); - this.leaderID = Preconditions.checkNotNull(leaderID); this.jobManagerGateway = Preconditions.checkNotNull(jobManagerGateway); } @@ -56,8 +51,8 @@ public ResourceID getJobManagerResourceID() { return jobManagerResourceID; } - public UUID getLeaderID() { - return leaderID; + public JobMasterId getJobMasterId() { + return jobManagerGateway.getFencingToken(); } public JobMasterGateway getJobManagerGateway() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java deleted file mode 100644 index d3ba9a97e8576..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/LeaderSessionIDException.java +++ /dev/null @@ -1,61 +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.rpc.exceptions; - -import java.util.UUID; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * An exception specifying that the received leader session ID is not the same as expected. - */ -public class LeaderSessionIDException extends Exception { - - private static final long serialVersionUID = -3276145308053264636L; - - /** expected leader session id */ - private final UUID expectedLeaderSessionID; - - /** actual leader session id */ - private final UUID actualLeaderSessionID; - - public LeaderSessionIDException(UUID expectedLeaderSessionID, UUID actualLeaderSessionID) { - super("Unmatched leader session ID : expected " + expectedLeaderSessionID + ", actual " + actualLeaderSessionID); - this.expectedLeaderSessionID = checkNotNull(expectedLeaderSessionID); - this.actualLeaderSessionID = checkNotNull(actualLeaderSessionID); - } - - /** - * Get expected leader session id - * - * @return expect leader session id - */ - public UUID getExpectedLeaderSessionID() { - return expectedLeaderSessionID; - } - - /** - * Get actual leader session id - * - * @return actual leader session id - */ - public UUID getActualLeaderSessionID() { - return actualLeaderSessionID; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderListener.java index f02a8c23694e7..65012a090af1d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderListener.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobLeaderListener.java @@ -21,8 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; import org.apache.flink.runtime.jobmaster.JobMasterGateway; - -import java.util.UUID; +import org.apache.flink.runtime.jobmaster.JobMasterId; /** * Listener for the {@link JobLeaderService}. The listener is notified whenever a job manager @@ -38,18 +37,17 @@ public interface JobLeaderListener { * * @param jobId identifying the job for which the job manager has gained leadership * @param jobManagerGateway to the job leader - * @param jobLeaderId new leader id of the job leader * @param registrationMessage containing further registration information */ - void jobManagerGainedLeadership(JobID jobId, JobMasterGateway jobManagerGateway, UUID jobLeaderId, JMTMRegistrationSuccess registrationMessage); + void jobManagerGainedLeadership(JobID jobId, JobMasterGateway jobManagerGateway, JMTMRegistrationSuccess registrationMessage); /** * Callback if the job leader for the job with the given job id lost its leadership. * * @param jobId identifying the job whose leader has lost leadership - * @param jobLeaderId old leader id + * @param jobMasterId old JobMasterId */ - void jobManagerLostLeadership(JobID jobId, UUID jobLeaderId); + void jobManagerLostLeadership(JobID jobId, JobMasterId jobMasterId); /** * Callback for errors which might occur in the {@link JobLeaderService}. 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 f564df4db1bda..20dcfa9321162 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 @@ -24,6 +24,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; 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.registration.RegisteredRpcConnection; @@ -32,6 +33,7 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.Preconditions; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,25 +56,25 @@ public class JobLeaderService { private static final Logger LOG = LoggerFactory.getLogger(JobLeaderService.class); - /** Self's location, used for the job manager connection */ + /** Self's location, used for the job manager connection. */ private final TaskManagerLocation ownLocation; - /** The leader retrieval service and listener for each registered job */ + /** The leader retrieval service and listener for each registered job. */ private final Map> jobLeaderServices; - /** Internal state of the service */ + /** Internal state of the service. */ private volatile JobLeaderService.State state; - /** Address of the owner of this service. This address is used for the job manager connection */ + /** Address of the owner of this service. This address is used for the job manager connection. */ private String ownerAddress; - /** Rpc service to use for establishing connections */ + /** Rpc service to use for establishing connections. */ private RpcService rpcService; - /** High availability services to create the leader retrieval services from */ + /** High availability services to create the leader retrieval services from. */ private HighAvailabilityServices highAvailabilityServices; - /** Job leader listener listening for job leader changes */ + /** Job leader listener listening for job leader changes. */ private JobLeaderListener jobLeaderListener; public JobLeaderService(TaskManagerLocation location) { @@ -207,24 +209,24 @@ public void addJob(final JobID jobId, final String defaultTargetAddress) throws */ private final class JobManagerLeaderListener implements LeaderRetrievalListener { - /** Job id identifying the job to look for a leader */ + /** Job id identifying the job to look for a leader. */ private final JobID jobId; - /** Rpc connection to the job leader */ - private RegisteredRpcConnection rpcConnection; + /** Rpc connection to the job leader. */ + private RegisteredRpcConnection rpcConnection; - /** State of the listener */ + /** State of the listener. */ private volatile boolean stopped; - /** Leader id of the current job leader */ - private volatile UUID currentLeaderId; + /** Leader id of the current job leader. */ + private volatile JobMasterId currentJobMasterId; private JobManagerLeaderListener(JobID jobId) { this.jobId = Preconditions.checkNotNull(jobId); stopped = false; rpcConnection = null; - currentLeaderId = null; + currentJobMasterId = null; } public void stop() { @@ -241,8 +243,10 @@ public void notifyLeaderAddress(final String leaderAddress, final UUID leaderId) LOG.debug("{}'s leader retrieval listener reported a new leader for job {}. " + "However, the service is no longer running.", JobLeaderService.class.getSimpleName(), jobId); } else { + final JobMasterId jobMasterId = leaderId != null ? new JobMasterId(leaderId) : null; + LOG.debug("New leader information for job {}. Address: {}, leader id: {}.", - jobId, leaderAddress, leaderId); + jobId, leaderAddress, jobMasterId); if (leaderAddress == null || leaderAddress.isEmpty()) { // the leader lost leadership but there is no other leader yet. @@ -250,28 +254,28 @@ public void notifyLeaderAddress(final String leaderAddress, final UUID leaderId) rpcConnection.close(); } - jobLeaderListener.jobManagerLostLeadership(jobId, currentLeaderId); + jobLeaderListener.jobManagerLostLeadership(jobId, currentJobMasterId); - currentLeaderId = leaderId; + currentJobMasterId = jobMasterId; } else { - currentLeaderId = leaderId; + currentJobMasterId = jobMasterId; if (rpcConnection != null) { // check if we are already trying to connect to this leader - if (!leaderId.equals(rpcConnection.getTargetLeaderId())) { + if (!Objects.equals(jobMasterId, rpcConnection.getTargetLeaderId())) { rpcConnection.close(); rpcConnection = new JobManagerRegisteredRpcConnection( LOG, leaderAddress, - leaderId, + jobMasterId, rpcService.getExecutor()); } } else { rpcConnection = new JobManagerRegisteredRpcConnection( LOG, leaderAddress, - leaderId, + jobMasterId, rpcService.getExecutor()); } @@ -300,18 +304,18 @@ public void handleError(Exception exception) { /** * Rpc connection for the job manager <--> task manager connection. */ - private final class JobManagerRegisteredRpcConnection extends RegisteredRpcConnection { + private final class JobManagerRegisteredRpcConnection extends RegisteredRpcConnection { JobManagerRegisteredRpcConnection( Logger log, String targetAddress, - UUID targetLeaderId, + JobMasterId jobMasterId, Executor executor) { - super(log, targetAddress, targetLeaderId, executor); + super(log, targetAddress, jobMasterId, executor); } @Override - protected RetryingRegistration generateRegistration() { + protected RetryingRegistration generateRegistration() { return new JobLeaderService.JobManagerRetryingRegistration( LOG, rpcService, @@ -326,10 +330,10 @@ protected RetryingRegistration @Override protected void onRegistrationSuccess(JMTMRegistrationSuccess success) { // filter out old registration attempts - if (Objects.equals(getTargetLeaderId(), currentLeaderId)) { + if (Objects.equals(getTargetLeaderId(), currentJobMasterId)) { log.info("Successful registration at job manager {} for job {}.", getTargetAddress(), jobId); - jobLeaderListener.jobManagerGainedLeadership(jobId, getTargetGateway(), getTargetLeaderId(), success); + jobLeaderListener.jobManagerGainedLeadership(jobId, getTargetGateway(), success); } else { log.debug("Encountered obsolete JobManager registration success from {} with leader session ID {}.", getTargetAddress(), getTargetLeaderId()); } @@ -338,7 +342,7 @@ protected void onRegistrationSuccess(JMTMRegistrationSuccess success) { @Override protected void onRegistrationFailure(Throwable failure) { // filter out old registration attempts - if (Objects.equals(getTargetLeaderId(), currentLeaderId)) { + if (Objects.equals(getTargetLeaderId(), currentJobMasterId)) { log.info("Failed to register at job manager {} for job {}.", getTargetAddress(), jobId); jobLeaderListener.handleError(failure); } else { @@ -352,7 +356,7 @@ protected void onRegistrationFailure(Throwable failure) { * Retrying registration for the job manager <--> task manager connection. */ private static final class JobManagerRetryingRegistration - extends RetryingRegistration + extends RetryingRegistration { private final String taskManagerRpcAddress; @@ -365,11 +369,10 @@ private static final class JobManagerRetryingRegistration String targetName, Class targetType, String targetAddress, - UUID leaderId, + JobMasterId jobMasterId, String taskManagerRpcAddress, - TaskManagerLocation taskManagerLocation) - { - super(log, rpcService, targetName, targetType, targetAddress, leaderId); + TaskManagerLocation taskManagerLocation) { + super(log, rpcService, targetName, targetType, targetAddress, jobMasterId); this.taskManagerRpcAddress = taskManagerRpcAddress; this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); @@ -377,15 +380,15 @@ private static final class JobManagerRetryingRegistration @Override protected CompletableFuture invokeRegistration( - JobMasterGateway gateway, UUID leaderId, long timeoutMillis) throws Exception - { - return gateway.registerTaskManager(taskManagerRpcAddress, taskManagerLocation, - leaderId, Time.milliseconds(timeoutMillis)); + JobMasterGateway gateway, + JobMasterId jobMasterId, + long timeoutMillis) throws Exception { + return gateway.registerTaskManager(taskManagerRpcAddress, taskManagerLocation, Time.milliseconds(timeoutMillis)); } } /** - * Internal state of the service + * Internal state of the service. */ private enum State { CREATED, STARTED, STOPPED diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java index 363c1072ef629..2c05388a673be 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/JobManagerConnection.java @@ -25,12 +25,11 @@ import org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.taskmanager.CheckpointResponder; import org.apache.flink.runtime.taskmanager.TaskManagerActions; import org.apache.flink.util.Preconditions; -import java.util.UUID; - /** * Container class for JobManager specific communication utils used by the {@link TaskExecutor}. */ @@ -42,9 +41,6 @@ public class JobManagerConnection { // The unique id used for identifying the job manager private final ResourceID resourceID; - // Job master leader session id - private final UUID leaderId; - // Gateway to the job master private final JobMasterGateway jobMasterGateway; @@ -70,7 +66,6 @@ public JobManagerConnection( JobID jobID, ResourceID resourceID, JobMasterGateway jobMasterGateway, - UUID leaderId, TaskManagerActions taskManagerActions, CheckpointResponder checkpointResponder, BlobCache blobCache, LibraryCacheManager libraryCacheManager, @@ -78,7 +73,6 @@ public JobManagerConnection( PartitionProducerStateChecker partitionStateChecker) { this.jobID = Preconditions.checkNotNull(jobID); this.resourceID = Preconditions.checkNotNull(resourceID); - this.leaderId = Preconditions.checkNotNull(leaderId); this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); this.taskManagerActions = Preconditions.checkNotNull(taskManagerActions); this.checkpointResponder = Preconditions.checkNotNull(checkpointResponder); @@ -96,8 +90,8 @@ public ResourceID getResourceID() { return resourceID; } - public UUID getLeaderId() { - return leaderId; + public JobMasterId getJobMasterId() { + return jobMasterGateway.getFencingToken(); } public JobMasterGateway getJobManagerGateway() { 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 3b1a1b49b72b6..b6a0637b0062c 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 @@ -51,14 +51,15 @@ import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; 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.memory.MemoryManager; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.metrics.MetricRegistry; -import org.apache.flink.runtime.registration.RegistrationConnectionListener; -import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.metrics.groups.TaskManagerMetricGroup; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; +import org.apache.flink.runtime.registration.RegistrationConnectionListener; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; @@ -302,7 +303,7 @@ public void postStop() throws Exception { @Override public CompletableFuture submitTask( TaskDeploymentDescriptor tdd, - UUID jobManagerLeaderId, + JobMasterId jobMasterId, Time timeout) { try { @@ -327,10 +328,10 @@ public CompletableFuture submitTask( throw new TaskSubmissionException(message); } - if (!Objects.equals(jobManagerConnection.getLeaderId(), jobManagerLeaderId)) { + if (!Objects.equals(jobManagerConnection.getJobMasterId(), jobMasterId)) { final String message = "Rejecting the task submission because the job manager leader id " + - jobManagerLeaderId + " does not match the expected job manager leader id " + - jobManagerConnection.getLeaderId() + '.'; + jobMasterId + " does not match the expected job manager leader id " + + jobManagerConnection.getJobMasterId() + '.'; log.debug(message); throw new TaskSubmissionException(message); @@ -353,9 +354,7 @@ public CompletableFuture submitTask( tdd.getAttemptNumber()); InputSplitProvider inputSplitProvider = new RpcInputSplitProvider( - jobManagerConnection.getLeaderId(), jobManagerConnection.getJobManagerGateway(), - jobInformation.getJobId(), taskInformation.getJobVertexId(), tdd.getExecutionAttemptId(), taskManagerConfiguration.getTimeout()); @@ -766,7 +765,7 @@ private void offerSlotsToJobManager(final JobID jobId) { final JobMasterGateway jobMasterGateway = jobManagerConnection.getJobManagerGateway(); final Iterator reservedSlotsIterator = taskSlotTable.getAllocatedSlots(jobId); - final UUID leaderId = jobManagerConnection.getLeaderId(); + final JobMasterId jobMasterId = jobManagerConnection.getJobMasterId(); final Collection reservedSlots = new HashSet<>(2); @@ -777,13 +776,11 @@ private void offerSlotsToJobManager(final JobID jobId) { // the slot is either free or releasing at the moment final String message = "Could not mark slot " + jobId + " active."; log.debug(message); - jobMasterGateway.failSlot(getResourceID(), offer.getAllocationId(), - leaderId, new Exception(message)); + jobMasterGateway.failSlot(getResourceID(), offer.getAllocationId(), new Exception(message)); } } catch (SlotNotFoundException e) { final String message = "Could not mark slot " + jobId + " active."; - jobMasterGateway.failSlot(getResourceID(), offer.getAllocationId(), - leaderId, new Exception(message)); + jobMasterGateway.failSlot(getResourceID(), offer.getAllocationId(), new Exception(message)); continue; } reservedSlots.add(offer); @@ -792,7 +789,6 @@ private void offerSlotsToJobManager(final JobID jobId) { CompletableFuture> acceptedSlotsFuture = jobMasterGateway.offerSlots( getResourceID(), reservedSlots, - leaderId, taskManagerConfiguration.getTimeout()); acceptedSlotsFuture.whenCompleteAsync( @@ -813,7 +809,7 @@ private void offerSlotsToJobManager(final JobID jobId) { } } else { // check if the response is still valid - if (isJobManagerConnectionValid(jobId, leaderId)) { + if (isJobManagerConnectionValid(jobId, jobMasterId)) { // mark accepted slots active for (SlotOffer acceptedSlot : acceptedSlots) { reservedSlots.remove(acceptedSlot); @@ -839,14 +835,14 @@ private void offerSlotsToJobManager(final JobID jobId) { } } - private void establishJobManagerConnection(JobID jobId, final JobMasterGateway jobMasterGateway, UUID jobManagerLeaderId, JMTMRegistrationSuccess registrationSuccess) { + private void establishJobManagerConnection(JobID jobId, final JobMasterGateway jobMasterGateway, JMTMRegistrationSuccess registrationSuccess) { if (jobManagerTable.contains(jobId)) { JobManagerConnection oldJobManagerConnection = jobManagerTable.get(jobId); - if (Objects.equals(oldJobManagerConnection.getLeaderId(), jobManagerLeaderId)) { + if (Objects.equals(oldJobManagerConnection.getJobMasterId(), jobMasterGateway.getFencingToken())) { // we already are connected to the given job manager - log.debug("Ignore JobManager gained leadership message for {} because we are already connected to it.", jobManagerLeaderId); + log.debug("Ignore JobManager gained leadership message for {} because we are already connected to it.", jobMasterGateway.getFencingToken()); return; } else { closeJobManagerConnection(jobId, new Exception("Found new job leader for job id " + jobId + '.')); @@ -860,7 +856,6 @@ private void establishJobManagerConnection(JobID jobId, final JobMasterGateway j jobId, jobManagerResourceID, jobMasterGateway, - jobManagerLeaderId, registrationSuccess.getBlobPort()); jobManagerConnections.put(jobManagerResourceID, newJobManagerConnection); jobManagerTable.put(jobId, newJobManagerConnection); @@ -927,15 +922,13 @@ private JobManagerConnection associateWithJobManager( JobID jobID, ResourceID resourceID, JobMasterGateway jobMasterGateway, - UUID jobManagerLeaderId, int blobPort) { Preconditions.checkNotNull(jobID); Preconditions.checkNotNull(resourceID); - Preconditions.checkNotNull(jobManagerLeaderId); Preconditions.checkNotNull(jobMasterGateway); Preconditions.checkArgument(blobPort > 0 || blobPort < MAX_BLOB_PORT, "Blob server port is out of range."); - TaskManagerActions taskManagerActions = new TaskManagerActionsImpl(jobManagerLeaderId, jobMasterGateway); + TaskManagerActions taskManagerActions = new TaskManagerActionsImpl(jobMasterGateway); CheckpointResponder checkpointResponder = new RpcCheckpointResponder(jobMasterGateway); @@ -959,18 +952,16 @@ private JobManagerConnection associateWithJobManager( } ResultPartitionConsumableNotifier resultPartitionConsumableNotifier = new RpcResultPartitionConsumableNotifier( - jobManagerLeaderId, jobMasterGateway, getRpcService().getExecutor(), taskManagerConfiguration.getTimeout()); - PartitionProducerStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobManagerLeaderId, jobMasterGateway); + PartitionProducerStateChecker partitionStateChecker = new RpcPartitionStateChecker(jobMasterGateway); return new JobManagerConnection( jobID, resourceID, jobMasterGateway, - jobManagerLeaderId, taskManagerActions, checkpointResponder, blobCache, @@ -1006,13 +997,12 @@ private void failTask(final ExecutionAttemptID executionAttemptID, final Throwab } private void updateTaskExecutionState( - final UUID jobMasterLeaderId, final JobMasterGateway jobMasterGateway, final TaskExecutionState taskExecutionState) { final ExecutionAttemptID executionAttemptID = taskExecutionState.getID(); - CompletableFuture futureAcknowledge = jobMasterGateway.updateTaskExecutionState(jobMasterLeaderId, taskExecutionState); + CompletableFuture futureAcknowledge = jobMasterGateway.updateTaskExecutionState(taskExecutionState); futureAcknowledge.whenCompleteAsync( (ack, throwable) -> { @@ -1024,7 +1014,6 @@ private void updateTaskExecutionState( } private void unregisterTaskAndNotifyFinalState( - final UUID jobMasterLeaderId, final JobMasterGateway jobMasterGateway, final ExecutionAttemptID executionAttemptID) { @@ -1044,7 +1033,6 @@ private void unregisterTaskAndNotifyFinalState( AccumulatorSnapshot accumulatorSnapshot = task.getAccumulatorRegistry().getSnapshot(); updateTaskExecutionState( - jobMasterLeaderId, jobMasterGateway, new TaskExecutionState( task.getJobID(), @@ -1101,10 +1089,10 @@ private boolean isConnectedToResourceManager() { return (resourceManagerConnection != null && resourceManagerConnection.isConnected()); } - private boolean isJobManagerConnectionValid(JobID jobId, UUID leaderId) { + private boolean isJobManagerConnectionValid(JobID jobId, JobMasterId jobMasterId) { JobManagerConnection jmConnection = jobManagerTable.get(jobId); - return jmConnection != null && Objects.equals(jmConnection.getLeaderId(), leaderId); + return jmConnection != null && Objects.equals(jmConnection.getJobMasterId(), jobMasterId); } // ------------------------------------------------------------------------ @@ -1152,7 +1140,7 @@ HeartbeatManager getResourceManagerHeartbeatManager() { // ------------------------------------------------------------------------ /** - * The listener for leader changes of the resource manager + * The listener for leader changes of the resource manager. */ private final class ResourceManagerLeaderListener implements LeaderRetrievalListener { @@ -1176,23 +1164,18 @@ private final class JobLeaderListenerImpl implements JobLeaderListener { public void jobManagerGainedLeadership( final JobID jobId, final JobMasterGateway jobManagerGateway, - final UUID jobLeaderId, final JMTMRegistrationSuccess registrationMessage) { - runAsync(new Runnable() { - @Override - public void run() { + runAsync( + () -> establishJobManagerConnection( jobId, jobManagerGateway, - jobLeaderId, - registrationMessage); - } - }); + registrationMessage)); } @Override - public void jobManagerLostLeadership(final JobID jobId, final UUID jobLeaderId) { - log.info("JobManager for job {} with leader id {} lost leadership.", jobId, jobLeaderId); + public void jobManagerLostLeadership(final JobID jobId, final JobMasterId jobMasterId) { + log.info("JobManager for job {} with leader id {} lost leadership.", jobId, jobMasterId); runAsync(new Runnable() { @Override @@ -1233,11 +1216,9 @@ public void onRegistrationFailure(Throwable failure) { } private final class TaskManagerActionsImpl implements TaskManagerActions { - private final UUID jobMasterLeaderId; private final JobMasterGateway jobMasterGateway; - private TaskManagerActionsImpl(UUID jobMasterLeaderId, JobMasterGateway jobMasterGateway) { - this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId); + private TaskManagerActionsImpl(JobMasterGateway jobMasterGateway) { this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); } @@ -1246,7 +1227,7 @@ public void notifyFinalState(final ExecutionAttemptID executionAttemptID) { runAsync(new Runnable() { @Override public void run() { - unregisterTaskAndNotifyFinalState(jobMasterLeaderId, jobMasterGateway, executionAttemptID); + unregisterTaskAndNotifyFinalState(jobMasterGateway, executionAttemptID); } }); } @@ -1273,7 +1254,7 @@ public void run() { @Override public void updateTaskExecutionState(final TaskExecutionState taskExecutionState) { - TaskExecutor.this.updateTaskExecutionState(jobMasterLeaderId, jobMasterGateway, taskExecutionState); + TaskExecutor.this.updateTaskExecutionState(jobMasterGateway, taskExecutionState); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java index fd5625550296e..ee0f69d234d4e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java @@ -27,13 +27,13 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.executiongraph.PartitionInfo; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcTimeout; import org.apache.flink.runtime.taskmanager.Task; -import java.util.UUID; import java.util.concurrent.CompletableFuture; /** @@ -64,13 +64,13 @@ CompletableFuture requestSlot( * Submit a {@link Task} to the {@link TaskExecutor}. * * @param tdd describing the task to submit - * @param leaderId of the job leader + * @param jobMasterId identifying the submitting JobMaster * @param timeout of the submit operation * @return Future acknowledge of the successful operation */ CompletableFuture submitTask( TaskDeploymentDescriptor tdd, - UUID leaderId, + JobMasterId jobMasterId, @RpcTimeout Time timeout); /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java index a919c7878bbab..baa403bc3722b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcInputSplitProvider.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.taskexecutor.rpc; -import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.core.io.InputSplit; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -30,27 +29,20 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; -import java.util.UUID; import java.util.concurrent.CompletableFuture; public class RpcInputSplitProvider implements InputSplitProvider { - private final UUID jobMasterLeaderId; private final JobMasterGateway jobMasterGateway; - private final JobID jobID; private final JobVertexID jobVertexID; private final ExecutionAttemptID executionAttemptID; private final Time timeout; public RpcInputSplitProvider( - UUID jobMasterLeaderId, JobMasterGateway jobMasterGateway, - JobID jobID, JobVertexID jobVertexID, ExecutionAttemptID executionAttemptID, Time timeout) { - this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId); this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); - this.jobID = Preconditions.checkNotNull(jobID); this.jobVertexID = Preconditions.checkNotNull(jobVertexID); this.executionAttemptID = Preconditions.checkNotNull(executionAttemptID); this.timeout = Preconditions.checkNotNull(timeout); @@ -62,7 +54,8 @@ public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) throws Inpu Preconditions.checkNotNull(userCodeClassLoader); CompletableFuture futureInputSplit = jobMasterGateway.requestNextInputSplit( - jobMasterLeaderId, jobVertexID, executionAttemptID); + jobVertexID, + executionAttemptID); try { SerializedInputSplit serializedInputSplit = futureInputSplit.get(timeout.getSize(), timeout.getUnit()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java index 26e1b0efa781c..f3eb717166a92 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcPartitionStateChecker.java @@ -26,16 +26,13 @@ import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.util.Preconditions; -import java.util.UUID; import java.util.concurrent.CompletableFuture; public class RpcPartitionStateChecker implements PartitionProducerStateChecker { - private final UUID jobMasterLeaderId; private final JobMasterGateway jobMasterGateway; - public RpcPartitionStateChecker(UUID jobMasterLeaderId, JobMasterGateway jobMasterGateway) { - this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId); + public RpcPartitionStateChecker(JobMasterGateway jobMasterGateway) { this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); } @@ -45,6 +42,6 @@ public CompletableFuture requestPartitionProducerState( IntermediateDataSetID resultId, ResultPartitionID partitionId) { - return jobMasterGateway.requestPartitionState(jobMasterLeaderId, resultId, partitionId); + return jobMasterGateway.requestPartitionState(resultId, partitionId); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java index d8985620bb828..82a6fbccbe863 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/rpc/RpcResultPartitionConsumableNotifier.java @@ -29,7 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -37,25 +36,21 @@ public class RpcResultPartitionConsumableNotifier implements ResultPartitionCons private static final Logger LOG = LoggerFactory.getLogger(RpcResultPartitionConsumableNotifier.class); - private final UUID jobMasterLeaderId; private final JobMasterGateway jobMasterGateway; private final Executor executor; private final Time timeout; public RpcResultPartitionConsumableNotifier( - UUID jobMasterLeaderId, JobMasterGateway jobMasterGateway, Executor executor, Time timeout) { - this.jobMasterLeaderId = Preconditions.checkNotNull(jobMasterLeaderId); this.jobMasterGateway = Preconditions.checkNotNull(jobMasterGateway); this.executor = Preconditions.checkNotNull(executor); this.timeout = Preconditions.checkNotNull(timeout); } @Override public void notifyPartitionConsumable(JobID jobId, ResultPartitionID partitionId, final TaskActions taskActions) { - CompletableFuture acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers( - jobMasterLeaderId, partitionId, timeout); + CompletableFuture acknowledgeFuture = jobMasterGateway.scheduleOrUpdateConsumers(partitionId, timeout); acknowledgeFuture.whenCompleteAsync( (Acknowledge ack, Throwable throwable) -> { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java index 55499f5aae025..6013e91e2cec5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.instance.ActorGateway; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; @@ -577,7 +578,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { final ResourceID jmResourceId = new ResourceID(jobMasterAddress); final ResourceID rmResourceId = ResourceID.generate(); final ResourceManagerId rmLeaderId = ResourceManagerId.generate(); - final UUID jmLeaderId = UUID.randomUUID(); + final JobMasterId jobMasterId = JobMasterId.generate(); final JobID jobId = new JobID(); final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); @@ -590,7 +591,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { Time.seconds(5L)); final TestingLeaderElectionService rmLeaderElectionService = new TestingLeaderElectionService(); - final TestingLeaderRetrievalService jmLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderId); + final TestingLeaderRetrievalService jmLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jobMasterId.toUUID()); final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); highAvailabilityServices.setResourceManagerLeaderElectionService(rmLeaderElectionService); highAvailabilityServices.setJobMasterLeaderRetriever(jobId, jmLeaderRetrievalService); @@ -633,7 +634,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { // test registration response successful and it will trigger monitor heartbeat target, schedule heartbeat request at interval time CompletableFuture successfulFuture = rmGateway.registerJobManager( - jmLeaderId, + jobMasterId, jmResourceId, jobMasterAddress, jobId, @@ -665,7 +666,7 @@ public void testHeartbeatTimeoutWithJobManager() throws Exception { // run the timeout runnable to simulate a heartbeat timeout timeoutRunnable.run(); - verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).disconnectResourceManager(eq(jmLeaderId), eq(rmLeaderId), any(TimeoutException.class)); + verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).disconnectResourceManager(eq(rmLeaderId), any(TimeoutException.class)); } finally { rpcService.stopService(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java index 8d613ac480f76..9d742e278fa44 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolRpcTest.java @@ -25,15 +25,16 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.util.clock.SystemClock; +import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -47,7 +48,7 @@ /** * Tests for the SlotPool using a proper RPC setup. */ -public class SlotPoolRpcTest { +public class SlotPoolRpcTest extends TestLogger { private static RpcService rpcService; @@ -80,7 +81,7 @@ public void testSlotAllocationNoResourceManager() throws Exception { Time.days(1), Time.days(1), Time.milliseconds(100) // this is the timeout for the request tested here ); - pool.start(UUID.randomUUID(), "foobar"); + pool.start(JobMasterId.generate(), "foobar"); CompletableFuture future = pool.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, Time.days(1)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java index ead453e485e33..5993dcbce53a2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotPoolTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit; import org.apache.flink.runtime.jobmanager.slots.AllocatedSlot; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.resourcemanager.SlotRequest; import org.apache.flink.runtime.rpc.RpcService; @@ -39,7 +40,6 @@ import org.mockito.Mockito; import java.util.List; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -88,7 +88,7 @@ public void testAllocateSimpleSlot() throws Exception { assertFalse(future.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(JobMasterId.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); @@ -125,7 +125,7 @@ public void testAllocationFulfilledByReturnedSlot() throws Exception { ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds()).times(2)) - .requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + .requestSlot(any(JobMasterId.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final List slotRequests = slotRequestArgumentCaptor.getAllValues(); @@ -168,7 +168,7 @@ public void testAllocateWithFreeSlot() throws Exception { assertFalse(future1.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(JobMasterId.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); @@ -211,7 +211,7 @@ public void testOfferSlot() throws Exception { assertFalse(future.isDone()); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(JobMasterId.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); @@ -266,7 +266,7 @@ public void returnAllocatedSlot(Slot slot) { CompletableFuture future1 = slotPoolGateway.allocateSlot(mock(ScheduledUnit.class), DEFAULT_TESTING_PROFILE, null, timeout); ArgumentCaptor slotRequestArgumentCaptor = ArgumentCaptor.forClass(SlotRequest.class); - verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(UUID.class), slotRequestArgumentCaptor.capture(), any(Time.class)); + verify(resourceManagerGateway, Mockito.timeout(timeout.toMilliseconds())).requestSlot(any(JobMasterId.class), slotRequestArgumentCaptor.capture(), any(Time.class)); final SlotRequest slotRequest = slotRequestArgumentCaptor.getValue(); @@ -297,7 +297,7 @@ public void returnAllocatedSlot(Slot slot) { private static ResourceManagerGateway createResourceManagerGatewayMock() { ResourceManagerGateway resourceManagerGateway = mock(ResourceManagerGateway.class); when(resourceManagerGateway - .requestSlot(any(UUID.class), any(SlotRequest.class), any(Time.class))) + .requestSlot(any(JobMasterId.class), any(SlotRequest.class), any(Time.class))) .thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); return resourceManagerGateway; @@ -308,7 +308,7 @@ private static SlotPoolGateway setupSlotPool( ResourceManagerGateway resourceManagerGateway) throws Exception { final String jobManagerAddress = "foobar"; - slotPool.start(UUID.randomUUID(), jobManagerAddress); + slotPool.start(JobMasterId.generate(), jobManagerAddress); slotPool.connectToResourceManager(resourceManagerGateway); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java index 0f38db2f1c890..b4f50fbb01668 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobManagerRunnerMockTest.java @@ -133,7 +133,7 @@ public void testStartAndShutdown() throws Exception { assertTrue(!jobCompletion.isJobFinished()); assertTrue(!jobCompletion.isJobFailed()); - verify(jobManager).start(any(UUID.class), any(Time.class)); + verify(jobManager).start(any(JobMasterId.class), any(Time.class)); runner.shutdown(); verify(leaderElectionService).stop(); @@ -165,9 +165,9 @@ public void testShutdownBeforeGrantLeadership() throws Exception { public void testJobFinished() throws Exception { runner.start(); - UUID leaderSessionID = UUID.randomUUID(); - runner.grantLeadership(leaderSessionID); - verify(jobManager).start(eq(leaderSessionID), any(Time.class)); + JobMasterId jobMasterId = JobMasterId.generate(); + runner.grantLeadership(jobMasterId.toUUID()); + verify(jobManager).start(eq(jobMasterId), any(Time.class)); assertTrue(!jobCompletion.isJobFinished()); // runner been told by JobManager that job is finished @@ -185,9 +185,9 @@ public void testJobFinished() throws Exception { public void testJobFailed() throws Exception { runner.start(); - UUID leaderSessionID = UUID.randomUUID(); - runner.grantLeadership(leaderSessionID); - verify(jobManager).start(eq(leaderSessionID), any(Time.class)); + JobMasterId jobMasterId = JobMasterId.generate(); + runner.grantLeadership(jobMasterId.toUUID()); + verify(jobManager).start(eq(jobMasterId), any(Time.class)); assertTrue(!jobCompletion.isJobFinished()); // runner been told by JobManager that job is failed @@ -204,9 +204,9 @@ public void testJobFailed() throws Exception { public void testLeadershipRevoked() throws Exception { runner.start(); - UUID leaderSessionID = UUID.randomUUID(); - runner.grantLeadership(leaderSessionID); - verify(jobManager).start(eq(leaderSessionID), any(Time.class)); + JobMasterId jobMasterId = JobMasterId.generate(); + runner.grantLeadership(jobMasterId.toUUID()); + verify(jobManager).start(eq(jobMasterId), any(Time.class)); assertTrue(!jobCompletion.isJobFinished()); runner.revokeLeadership(); @@ -219,18 +219,18 @@ public void testLeadershipRevoked() throws Exception { public void testRegainLeadership() throws Exception { runner.start(); - UUID leaderSessionID = UUID.randomUUID(); - runner.grantLeadership(leaderSessionID); - verify(jobManager).start(eq(leaderSessionID), any(Time.class)); + JobMasterId jobMasterId = JobMasterId.generate(); + runner.grantLeadership(jobMasterId.toUUID()); + verify(jobManager).start(eq(jobMasterId), any(Time.class)); assertTrue(!jobCompletion.isJobFinished()); runner.revokeLeadership(); verify(jobManager).suspend(any(Throwable.class), any(Time.class)); assertFalse(runner.isShutdown()); - UUID leaderSessionID2 = UUID.randomUUID(); - runner.grantLeadership(leaderSessionID2); - verify(jobManager).start(eq(leaderSessionID2), any(Time.class)); + JobMasterId jobMasterId2 = JobMasterId.generate(); + runner.grantLeadership(jobMasterId2.toUUID()); + verify(jobManager).start(eq(jobMasterId2), any(Time.class)); } private static class TestingOnCompletionActions implements OnCompletionActions, FatalErrorHandler { 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 6282ea0320575..64cc13b624094 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 @@ -51,16 +51,12 @@ import java.net.InetAddress; import java.net.URL; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.*; -import static org.mockito.Mockito.when; @RunWith(PowerMockRunner.class) @PrepareForTest(BlobLibraryCacheManager.class) @@ -79,7 +75,7 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final String jobManagerAddress = "jm"; - final UUID jmLeaderId = UUID.randomUUID(); + final JobMasterId jobMasterId = JobMasterId.generate(); final ResourceID jmResourceId = new ResourceID(jobManagerAddress); final String taskManagerAddress = "tm"; @@ -118,7 +114,7 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { testingFatalErrorHandler, new FlinkUserCodeClassLoader(new URL[0])); - CompletableFuture startFuture = jobMaster.start(jmLeaderId, testingTimeout); + CompletableFuture startFuture = jobMaster.start(jobMasterId, testingTimeout); // wait for the start to complete startFuture.get(testingTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); @@ -127,7 +123,7 @@ public void testHeartbeatTimeoutWithTaskManager() throws Exception { // register task manager will trigger monitor heartbeat target, schedule heartbeat request at interval time CompletableFuture registrationResponse = jobMasterGateway - .registerTaskManager(taskManagerAddress, taskManagerLocation, jmLeaderId, testingTimeout); + .registerTaskManager(taskManagerAddress, taskManagerLocation, testingTimeout); // wait for the completion of the registration registrationResponse.get(); @@ -169,7 +165,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { final String resourceManagerAddress = "rm"; final String jobManagerAddress = "jm"; final ResourceManagerId resourceManagerId = ResourceManagerId.generate(); - final UUID jmLeaderId = UUID.randomUUID(); + final JobMasterId jobMasterId = JobMasterId.generate(); final ResourceID rmResourceId = new ResourceID(resourceManagerAddress); final ResourceID jmResourceId = new ResourceID(jobManagerAddress); final JobGraph jobGraph = new JobGraph(); @@ -188,7 +184,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { final ResourceManagerGateway resourceManagerGateway = mock(ResourceManagerGateway.class); when(resourceManagerGateway.registerJobManager( - any(UUID.class), + any(JobMasterId.class), any(ResourceID.class), anyString(), any(JobID.class), @@ -219,7 +215,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { testingFatalErrorHandler, new FlinkUserCodeClassLoader(new URL[0])); - CompletableFuture startFuture = jobMaster.start(jmLeaderId, testingTimeout); + CompletableFuture startFuture = jobMaster.start(jobMasterId, testingTimeout); // wait for the start operation to complete startFuture.get(testingTimeout.toMilliseconds(), TimeUnit.MILLISECONDS); @@ -229,7 +225,7 @@ public void testHeartbeatTimeoutWithResourceManager() throws Exception { // register job manager success will trigger monitor heartbeat target between jm and rm verify(resourceManagerGateway, timeout(testingTimeout.toMilliseconds())).registerJobManager( - eq(jmLeaderId), + eq(jobMasterId), eq(jmResourceId), anyString(), eq(jobGraph.getJobID()), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java index 7b8703e768be6..fb5ee8bad7ea7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -62,7 +63,7 @@ public class JobLeaderIdServiceTest extends TestLogger { public void testAddingJob() throws Exception { final JobID jobId = new JobID(); final String address = "foobar"; - final UUID leaderId = UUID.randomUUID(); + final JobMasterId leaderId = JobMasterId.generate(); TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService( null, @@ -83,10 +84,10 @@ public void testAddingJob() throws Exception { jobLeaderIdService.addJob(jobId); - CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); + CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); // notify the leader id service about the new leader - leaderRetrievalService.notifyListener(address, leaderId); + leaderRetrievalService.notifyListener(address, leaderId.toUUID()); assertEquals(leaderId, leaderIdFuture.get()); @@ -117,7 +118,7 @@ public void testRemovingJob() throws Exception { jobLeaderIdService.addJob(jobId); - CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); + CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); // remove the job before we could find a leader jobLeaderIdService.removeJob(jobId); @@ -183,7 +184,7 @@ public void testInitialJobTimeout() throws Exception { public void jobTimeoutAfterLostLeadership() throws Exception { final JobID jobId = new JobID(); final String address = "foobar"; - final UUID leaderId = UUID.randomUUID(); + final JobMasterId leaderId = JobMasterId.generate(); TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); TestingLeaderRetrievalService leaderRetrievalService = new TestingLeaderRetrievalService( null, @@ -228,10 +229,10 @@ public Object answer(InvocationOnMock invocation) throws Throwable { jobLeaderIdService.addJob(jobId); - CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); + CompletableFuture leaderIdFuture = jobLeaderIdService.getLeaderId(jobId); // notify the leader id service about the new leader - leaderRetrievalService.notifyListener(address, leaderId); + leaderRetrievalService.notifyListener(address, leaderId.toUUID()); assertEquals(leaderId, leaderIdFuture.get()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java index 1de32841ec723..156bc73ca3972 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -46,7 +47,6 @@ import org.junit.Before; import org.junit.Test; -import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -78,16 +78,16 @@ public void teardown() throws Exception { public void testRegisterJobMaster() throws Exception { String jobMasterAddress = "/jobMasterAddress1"; JobID jobID = mockJobMaster(jobMasterAddress); - UUID jmLeaderID = UUID.randomUUID(); + JobMasterId jobMasterId = JobMasterId.generate(); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); - TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jobMasterId.toUUID()); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final ResourceManager resourceManager = createAndStartResourceManager(mock(LeaderElectionService.class), jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final ResourceManagerGateway rmGateway = resourceManager.getSelfGateway(ResourceManagerGateway.class); // test response successful CompletableFuture successfulFuture = rmGateway.registerJobManager( - jmLeaderID, + jobMasterId, jmResourceId, jobMasterAddress, jobID, @@ -107,9 +107,9 @@ public void testRegisterJobMaster() throws Exception { public void testRegisterJobMasterWithUnmatchedLeaderSessionId1() throws Exception { String jobMasterAddress = "/jobMasterAddress1"; JobID jobID = mockJobMaster(jobMasterAddress); - UUID jmLeaderID = UUID.randomUUID(); + JobMasterId jobMasterId = JobMasterId.generate(); final ResourceID jmResourceId = new ResourceID(jobMasterAddress); - TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jmLeaderID); + TestingLeaderRetrievalService jobMasterLeaderRetrievalService = new TestingLeaderRetrievalService(jobMasterAddress, jobMasterId.toUUID()); TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); final ResourceManager resourceManager = createAndStartResourceManager(mock(LeaderElectionService.class), jobID, jobMasterLeaderRetrievalService, testingFatalErrorHandler); final ResourceManagerGateway wronglyFencedGateway = rpcService.connect(resourceManager.getAddress(), ResourceManagerId.generate(), ResourceManagerGateway.class) @@ -117,7 +117,7 @@ public void testRegisterJobMasterWithUnmatchedLeaderSessionId1() throws Exceptio // test throw exception when receive a registration from job master which takes unmatched leaderSessionId CompletableFuture unMatchedLeaderFuture = wronglyFencedGateway.registerJobManager( - jmLeaderID, + jobMasterId, jmResourceId, jobMasterAddress, jobID, @@ -152,9 +152,9 @@ public void testRegisterJobMasterWithUnmatchedLeaderSessionId2() throws Exceptio final ResourceID jmResourceId = new ResourceID(jobMasterAddress); // test throw exception when receive a registration from job master which takes unmatched leaderSessionId - UUID differentLeaderSessionID = UUID.randomUUID(); + JobMasterId differentJobMasterId = JobMasterId.generate(); CompletableFuture unMatchedLeaderFuture = rmGateway.registerJobManager( - differentLeaderSessionID, + differentJobMasterId, jmResourceId, jobMasterAddress, jobID, @@ -185,7 +185,7 @@ public void testRegisterJobMasterFromInvalidAddress() throws Exception { // test throw exception when receive a registration from job master which takes invalid address String invalidAddress = "/jobMasterAddress2"; CompletableFuture invalidAddressFuture = rmGateway.registerJobManager( - HighAvailabilityServices.DEFAULT_LEADER_ID, + new JobMasterId(HighAvailabilityServices.DEFAULT_LEADER_ID), jmResourceId, invalidAddress, jobID, @@ -221,7 +221,7 @@ public void testRegisterJobMasterWithFailureLeaderListener() throws Exception { // this should fail because we try to register a job leader listener for an unknown job id CompletableFuture registrationFuture = rmGateway.registerJobManager( - HighAvailabilityServices.DEFAULT_LEADER_ID, + new JobMasterId(HighAvailabilityServices.DEFAULT_LEADER_ID), jmResourceId, jobMasterAddress, unknownJobIDToHAServices, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index 945cbf36ff2cf..348dce647e023 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.jobmaster.JobMasterRegistrationSuccess; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; @@ -92,7 +93,7 @@ public void testSlotAllocation() throws Exception { final TestingLeaderRetrievalService rmLeaderRetrievalService = new TestingLeaderRetrievalService(null, null); final String rmAddress = "rm"; final String jmAddress = "jm"; - final UUID jmLeaderId = UUID.randomUUID(); + final JobMasterId jobMasterId = JobMasterId.generate(); final ResourceID rmResourceId = new ResourceID(rmAddress); final ResourceID jmResourceId = new ResourceID(jmAddress); final JobID jobId = new JobID(); @@ -100,7 +101,7 @@ public void testSlotAllocation() throws Exception { testingHAServices.setResourceManagerLeaderElectionService(rmLeaderElectionService); testingHAServices.setResourceManagerLeaderRetriever(rmLeaderRetrievalService); - testingHAServices.setJobMasterLeaderRetriever(jobId, new TestingLeaderRetrievalService(jmAddress, jmLeaderId)); + testingHAServices.setJobMasterLeaderRetriever(jobId, new TestingLeaderRetrievalService(jmAddress, jobMasterId.toUUID())); TestingRpcService rpcService = new TestingRpcService(); ResourceManagerConfiguration resourceManagerConfiguration = new ResourceManagerConfiguration( @@ -162,14 +163,14 @@ public void testSlotAllocation() throws Exception { JobMasterGateway jmGateway = mock(JobMasterGateway.class); - when(jmGateway.registerTaskManager(any(String.class), any(TaskManagerLocation.class), eq(jmLeaderId), any(Time.class))) + when(jmGateway.registerTaskManager(any(String.class), any(TaskManagerLocation.class), any(Time.class))) .thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(taskManagerResourceId, 1234))); when(jmGateway.getHostname()).thenReturn(jmAddress); when(jmGateway.offerSlots( eq(taskManagerResourceId), any(Iterable.class), - eq(jmLeaderId), any(Time.class))).thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); + when(jmGateway.getFencingToken()).thenReturn(jobMasterId); rpcService.registerGateway(rmAddress, resourceManager.getSelfGateway(ResourceManagerGateway.class)); @@ -193,7 +194,7 @@ public void testSlotAllocation() throws Exception { rmLeaderRetrievalService.notifyListener(rmAddress, rmLeaderId); CompletableFuture registrationResponseFuture = rmGateway.registerJobManager( - jmLeaderId, + jobMasterId, jmResourceId, jmAddress, jobId, @@ -203,14 +204,14 @@ public void testSlotAllocation() throws Exception { assertTrue(registrationResponse instanceof JobMasterRegistrationSuccess); - CompletableFuture slotAck = rmGateway.requestSlot(jmLeaderId, slotRequest, timeout); + CompletableFuture slotAck = rmGateway.requestSlot(jobMasterId, slotRequest, timeout); slotAck.get(); verify(jmGateway, Mockito.timeout(timeout.toMilliseconds())).offerSlots( eq(taskManagerResourceId), (Iterable)argThat(Matchers.contains(slotOffer)), - eq(jmLeaderId), any(Time.class)); + any(Time.class)); } finally { if (testingFatalErrorHandler.hasExceptionOccurred()) { testingFatalErrorHandler.rethrowError(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 2112c1bd4021a..714644514f166 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -55,6 +55,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmaster.JMTMRegistrationSuccess; import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.memory.MemoryManager; @@ -190,7 +191,6 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro when(jobMasterGateway.registerTaskManager( any(String.class), eq(taskManagerLocation), - eq(jmLeaderId), any(Time.class) )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getAddress()).thenReturn(jobMasterAddress); @@ -228,7 +228,7 @@ public HeartbeatManagerImpl answer(InvocationOnMock invocation) thro // register task manager success will trigger monitoring heartbeat target between tm and jm verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).registerTaskManager( - eq(taskManager.getAddress()), eq(taskManagerLocation), eq(jmLeaderId), any(Time.class)); + eq(taskManager.getAddress()), eq(taskManagerLocation), any(Time.class)); // the timeout should trigger disconnecting from the JobManager verify(jobMasterGateway, timeout(heartbeatTimeout * 50L)).disconnectTaskManager(eq(taskManagerLocation.getResourceID()), any(TimeoutException.class)); @@ -657,7 +657,7 @@ public void testTaskSubmission() throws Exception { final TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); final JobID jobId = new JobID(); final AllocationID allocationId = new AllocationID(); - final UUID jobManagerLeaderId = UUID.randomUUID(); + final JobMasterId jobMasterId = JobMasterId.generate(); final JobVertexID jobVertexId = new JobVertexID(); JobInformation jobInformation = new JobInformation( @@ -694,11 +694,13 @@ public void testTaskSubmission() throws Exception { final LibraryCacheManager libraryCacheManager = mock(LibraryCacheManager.class); when(libraryCacheManager.getClassLoader(any(JobID.class))).thenReturn(ClassLoader.getSystemClassLoader()); + final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); + when(jobMasterGateway.getFencingToken()).thenReturn(jobMasterId); + final JobManagerConnection jobManagerConnection = new JobManagerConnection( jobId, ResourceID.generate(), - mock(JobMasterGateway.class), - jobManagerLeaderId, + jobMasterGateway, mock(TaskManagerActions.class), mock(CheckpointResponder.class), mock(BlobCache.class), @@ -755,7 +757,7 @@ public void testTaskSubmission() throws Exception { final TaskExecutorGateway tmGateway = taskManager.getSelfGateway(TaskExecutorGateway.class); - tmGateway.submitTask(tdd, jobManagerLeaderId, timeout); + tmGateway.submitTask(tdd, jobMasterId, timeout); CompletableFuture completionFuture = TestInvokable.completableFuture; @@ -833,14 +835,12 @@ public void testJobLeaderDetection() throws Exception { when(jobMasterGateway.registerTaskManager( any(String.class), eq(taskManagerLocation), - eq(jobManagerLeaderId), any(Time.class) )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); when(jobMasterGateway.offerSlots( any(ResourceID.class), any(Iterable.class), - any(UUID.class), any(Time.class))).thenReturn(mock(CompletableFuture.class, RETURNS_MOCKS)); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); @@ -894,7 +894,6 @@ public void testJobLeaderDetection() throws Exception { verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).offerSlots( any(ResourceID.class), (Iterable)Matchers.argThat(contains(slotOffer)), - eq(jobManagerLeaderId), any(Time.class)); // check if a concurrent error occurred @@ -958,13 +957,12 @@ public void testSlotAcceptance() throws Exception { when(jobMasterGateway.registerTaskManager( any(String.class), eq(taskManagerLocation), - eq(jobManagerLeaderId), any(Time.class) )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); when(jobMasterGateway.offerSlots( - any(ResourceID.class), any(Iterable.class), eq(jobManagerLeaderId), any(Time.class))) + any(ResourceID.class), any(Iterable.class), any(Time.class))) .thenReturn(CompletableFuture.completedFuture((Collection)Collections.singleton(offer1))); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); @@ -1163,10 +1161,10 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { final ResourceID resourceManagerResourceId = new ResourceID(resourceManagerAddress); final String jobManagerAddress = "jm"; - final UUID jobManagerLeaderId = UUID.randomUUID(); + final JobMasterId jobMasterId = JobMasterId.generate(); final LeaderRetrievalService resourceManagerLeaderRetrievalService = new TestingLeaderRetrievalService(resourceManagerAddress, resourceManagerLeaderId); - final LeaderRetrievalService jobManagerLeaderRetrievalService = new TestingLeaderRetrievalService(jobManagerAddress, jobManagerLeaderId); + final LeaderRetrievalService jobManagerLeaderRetrievalService = new TestingLeaderRetrievalService(jobManagerAddress, jobMasterId.toUUID()); haServices.setResourceManagerLeaderRetriever(resourceManagerLeaderRetrievalService); haServices.setJobMasterLeaderRetriever(jobId, jobManagerLeaderRetrievalService); @@ -1193,13 +1191,10 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { when(jobMasterGateway.registerTaskManager( any(String.class), eq(taskManagerLocation), - eq(jobManagerLeaderId), any(Time.class) )).thenReturn(CompletableFuture.completedFuture(new JMTMRegistrationSuccess(jmResourceId, blobPort))); when(jobMasterGateway.getHostname()).thenReturn(jobManagerAddress); - when(jobMasterGateway.updateTaskExecutionState( - any(UUID.class), - any(TaskExecutionState.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); + when(jobMasterGateway.updateTaskExecutionState(any(TaskExecutionState.class))).thenReturn(CompletableFuture.completedFuture(Acknowledge.get())); rpc.registerGateway(resourceManagerAddress, resourceManagerGateway); @@ -1212,7 +1207,6 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { jobId, jmResourceId, jobMasterGateway, - jobManagerLeaderId, mock(TaskManagerActions.class), mock(CheckpointResponder.class), mock(BlobCache.class), @@ -1297,7 +1291,6 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { jobMasterGateway.offerSlots( any(ResourceID.class), any(Iterable.class), - eq(jobManagerLeaderId), any(Time.class))) .thenReturn(offerResultFuture); @@ -1305,10 +1298,10 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { // been properly started. This will also offer the slots to the job master jobLeaderService.addJob(jobId, jobManagerAddress); - verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).offerSlots(any(ResourceID.class), any(Iterable.class), eq(jobManagerLeaderId), any(Time.class)); + verify(jobMasterGateway, Mockito.timeout(timeout.toMilliseconds())).offerSlots(any(ResourceID.class), any(Iterable.class), any(Time.class)); // submit the task without having acknowledge the offered slots - tmGateway.submitTask(tdd, jobManagerLeaderId, timeout); + tmGateway.submitTask(tdd, jobMasterId, timeout); // acknowledge the offered slots offerResultFuture.complete(Collections.singleton(offer1)); @@ -1351,7 +1344,6 @@ public void testFilterOutDuplicateJobMasterRegistrations() throws Exception { final JobID jobId = new JobID(); final JobMasterGateway jobMasterGateway = mock(JobMasterGateway.class); when(jobMasterGateway.getHostname()).thenReturn("localhost"); - final UUID jobLeaderId = UUID.randomUUID(); final JMTMRegistrationSuccess registrationMessage = new JMTMRegistrationSuccess(ResourceID.generate(), 1); final JobManagerTable jobManagerTableMock = spy(new JobManagerTable()); @@ -1382,10 +1374,10 @@ public void testFilterOutDuplicateJobMasterRegistrations() throws Exception { JobLeaderListener taskExecutorListener = jobLeaderListenerArgumentCaptor.getValue(); - taskExecutorListener.jobManagerGainedLeadership(jobId, jobMasterGateway, jobLeaderId, registrationMessage); + taskExecutorListener.jobManagerGainedLeadership(jobId, jobMasterGateway, registrationMessage); // duplicate job manager gained leadership message - taskExecutorListener.jobManagerGainedLeadership(jobId, jobMasterGateway, jobLeaderId, registrationMessage); + taskExecutorListener.jobManagerGainedLeadership(jobId, jobMasterGateway, registrationMessage); ArgumentCaptor jobManagerConnectionArgumentCaptor = ArgumentCaptor.forClass(JobManagerConnection.class); From 09344aa2dc36b9b3ea4c5b7573ff532e26f9b0dd Mon Sep 17 00:00:00 2001 From: shaoxuan-wang Date: Wed, 16 Aug 2017 00:00:12 +0800 Subject: [PATCH 091/129] [FLINK-6751] [docs] Add documentation for user-defined AggregateFunction. This closes #4546. --- docs/dev/table/udfs.md | 408 +++++++++++++++++- docs/fig/udagg-mechanism.png | Bin 0 -> 201262 bytes .../table/functions/AggregateFunction.scala | 2 +- 3 files changed, 403 insertions(+), 7 deletions(-) create mode 100644 docs/fig/udagg-mechanism.png diff --git a/docs/dev/table/udfs.md b/docs/dev/table/udfs.md index 55f58b6032c68..6c9bc1af76d37 100644 --- a/docs/dev/table/udfs.md +++ b/docs/dev/table/udfs.md @@ -24,15 +24,18 @@ under the License. User-defined functions are an important feature, because they significantly extend the expressiveness of queries. -**TODO** - * This will be replaced by the TOC {:toc} Register User-Defined Functions ------------------------------- +In most cases, a user-defined function must be registered before it can be used in an query. It is not necessary to register functions for the Scala Table API. + +Functions are registered at the `TableEnvironment` by calling a `registerFunction()` method. When a user-defined function is registered, it is inserted into the function catalog of the `TableEnvironment` such that the Table API or SQL parser can recognize and properly translate it. + +Please find detailed examples of how to register and how to call each type of user-defined function +(`ScalarFunction`, `TableFunction`, and `AggregateFunction`) in the following sub-sessions. -**TODO** {% top %} @@ -97,8 +100,6 @@ tableEnv.sql("SELECT string, HASHCODE(string) FROM MyTable"); By default the result type of an evaluation method is determined by Flink's type extraction facilities. This is sufficient for basic types or simple POJOs but might be wrong for more complex, custom, or composite types. In these cases `TypeInformation` of the result type can be manually defined by overriding `ScalarFunction#getResultType()`. -Internally, the Table API and SQL code generation works with primitive values as much as possible. If a user-defined scalar function should not introduce much overhead through object creation/casting during runtime, it is recommended to declare parameters and result types as primitive types instead of their boxed classes. `Types.DATE` and `Types.TIME` can also be represented as `int`. `Types.TIMESTAMP` can be represented as `long`. - The following example shows an advanced example which takes the internal timestamp representation and also returns the internal timestamp representation as a long value. By overriding `ScalarFunction#getResultType()` we define that the returned long value should be interpreted as a `Types.TIMESTAMP` by the code generation.
@@ -264,10 +265,405 @@ class CustomTypeSplit extends TableFunction[Row] { {% top %} + Aggregation Functions --------------------- -**TODO** +User-Defined Aggregate Functions (UDAGGs) aggregate a table (one ore more rows with one or more attributes) to a scalar value. + +
+UDAGG mechanism +
+ +The above figure shows an example of an aggregation. Assume you have a table that contains data about beverages. The table consists of three columns, `id`, `name` and `price` and 5 rows. Imagine you need to find the highest price of all beverages in the table, i.e., perform a `max()` aggregation. You would need to check each of the 5 rows and the result would be a single numeric value. + +User-defined aggregation functions are implemented by extending the `AggregateFunction` class. An `AggregateFunction` works as follows. First, it needs an `accumulator`, which is the data structure that holds the intermediate result of the aggregation. An empty accumulator is created by calling the `createAccumulator()` method of the `AggregateFunction`. Subsequently, the `accumulate()` method of the function is called for each input row to update the accumulator. Once all rows have been processed, the `getValue()` method of the function is called to compute and return the final result. + +**The following methods are mandatory for each `AggregateFunction`:** + +- `createAccumulator()` +- `accumulate()` +- `getValue()` + +Flink’s type extraction facilities can fail to identify complex data types, e.g., if they are not basic types or simple POJOs. So similar to `ScalarFunction` and `TableFunction`, `AggregateFunction` provides methods to specify the `TypeInformation` of the result type (through + `AggregateFunction#getResultType()`) and the type of the accumulator (through `AggregateFunction#getAccumulatorType()`). + +Besides the above methods, there are a few contracted methods that can be +optionally implemented. While some of these methods allow the system more efficient query execution, others are mandatory for certain use cases. For instance, the `merge()` method is mandatory if the aggregation function should be applied in the context of a session group window (the accumulators of two session windows need to be joined when a row is observed that "connects" them). + +**The following methods of `AggregateFunction` are required depending on the use case:** + +- `retract()` is required for aggregations on bounded `OVER` windows. +- `merge()` is required for many batch aggreagtions and session window aggregations. +- `resetAccumulator()` is required for many batch aggregations. + +All methods of `AggregateFunction` must be declared as `public`, not `static` and named exactly as the names mentioned above. The methods `createAccumulator`, `getValue`, `getResultType`, and `getAccumulatorType` are defined in the `AggregateFunction` abstract class, while others are contracted methods. In order to define a table function, one has to extend the base class `org.apache.flink.table.functions.AggregateFunction` and implement one (or more) `accumulate` methods. + +Detailed documentation for all methods of `AggregateFunction` is given below. + +
+
+{% highlight java %} +/** + * Base class for aggregation functions. + * + * @param the type of the aggregation result + * @param the type of the aggregation accumulator. The accumulator is used to keep the + * aggregated values which are needed to compute an aggregation result. + * AggregateFunction represents its state using accumulator, thereby the state of the + * AggregateFunction must be put into the accumulator. + */ +public abstract class AggregateFunction extends UserDefinedFunction { + + /** + * Creates and init the Accumulator for this [[AggregateFunction]]. + * + * @return the accumulator with the initial value + */ + public ACC createAccumulator(); // MANDATORY + + /** Processes the input values and update the provided accumulator instance. The method + * accumulate can be overloaded with different custom types and arguments. An AggregateFunction + * requires at least one accumulate() method. + * + * @param accumulator the accumulator which contains the current aggregated results + * @param [user defined inputs] the input value (usually obtained from a new arrived data). + */ + public void accumulate(ACC accumulator, [user defined inputs]); // MANDATORY + + /** + * Retracts the input values from the accumulator instance. The current design assumes the + * inputs are the values that have been previously accumulated. The method retract can be + * overloaded with different custom types and arguments. This function must be implemented for + * datastream bounded over aggregate. + * + * @param accumulator the accumulator which contains the current aggregated results + * @param [user defined inputs] the input value (usually obtained from a new arrived data). + */ + public void retract(ACC accumulator, [user defined inputs]); // OPTIONAL + + /** + * Merges a group of accumulator instances into one accumulator instance. This function must be + * implemented for datastream session window grouping aggregate and dataset grouping aggregate. + * + * @param accumulator the accumulator which will keep the merged aggregate results. It should + * be noted that the accumulator may contain the previous aggregated + * results. Therefore user should not replace or clean this instance in the + * custom merge method. + * @param its an [[java.lang.Iterable]] pointed to a group of accumulators that will be + * merged. + */ + public void merge(ACC accumulator, java.lang.Iterable its); // OPTIONAL + + /** + * Called every time when an aggregation result should be materialized. + * The returned value could be either an early and incomplete result + * (periodically emitted as data arrive) or the final result of the + * aggregation. + * + * @param accumulator the accumulator which contains the current + * aggregated results + * @return the aggregation result + */ + public T getValue(ACC accumulator); // MANDATORY + + /** + * Resets the accumulator for this [[AggregateFunction]]. This function must be implemented for + * dataset grouping aggregate. + * + * @param accumulator the accumulator which needs to be reset + */ + public void resetAccumulator(ACC accumulator); // OPTIONAL + + /** + * Returns true if this AggregateFunction can only be applied in an OVER window. + * + * @return true if the AggregateFunction requires an OVER window, false otherwise. + */ + public Boolean requiresOver = false; // PRE-DEFINED + + /** + * Returns the TypeInformation of the AggregateFunction's result. + * + * @return The TypeInformation of the AggregateFunction's result or null if the result type + * should be automatically inferred. + */ + public TypeInformation getResultType = null; // PRE-DEFINED + + /** + * Returns the TypeInformation of the AggregateFunction's accumulator. + * + * @return The TypeInformation of the AggregateFunction's accumulator or null if the + * accumulator type should be automatically inferred. + */ + public TypeInformation getAccumulatorType = null; // PRE-DEFINED +} +{% endhighlight %} +
+ +
+{% highlight scala %} +/** + * Base class for aggregation functions. + * + * @tparam T the type of the aggregation result + * @tparam ACC the type of the aggregation accumulator. The accumulator is used to keep the + * aggregated values which are needed to compute an aggregation result. + * AggregateFunction represents its state using accumulator, thereby the state of the + * AggregateFunction must be put into the accumulator. + */ +abstract class AggregateFunction[T, ACC] extends UserDefinedFunction { + /** + * Creates and init the Accumulator for this [[AggregateFunction]]. + * + * @return the accumulator with the initial value + */ + def createAccumulator(): ACC // MANDATORY + + /** + * Processes the input values and update the provided accumulator instance. The method + * accumulate can be overloaded with different custom types and arguments. An AggregateFunction + * requires at least one accumulate() method. + * + * @param accumulator the accumulator which contains the current aggregated results + * @param [user defined inputs] the input value (usually obtained from a new arrived data). + */ + def accumulate(accumulator: ACC, [user defined inputs]): Unit // MANDATORY + + /** + * Retracts the input values from the accumulator instance. The current design assumes the + * inputs are the values that have been previously accumulated. The method retract can be + * overloaded with different custom types and arguments. This function must be implemented for + * datastream bounded over aggregate. + * + * @param accumulator the accumulator which contains the current aggregated results + * @param [user defined inputs] the input value (usually obtained from a new arrived data). + */ + def retract(accumulator: ACC, [user defined inputs]): Unit // OPTIONAL + + /** + * Merges a group of accumulator instances into one accumulator instance. This function must be + * implemented for datastream session window grouping aggregate and dataset grouping aggregate. + * + * @param accumulator the accumulator which will keep the merged aggregate results. It should + * be noted that the accumulator may contain the previous aggregated + * results. Therefore user should not replace or clean this instance in the + * custom merge method. + * @param its an [[java.lang.Iterable]] pointed to a group of accumulators that will be + * merged. + */ + def merge(accumulator: ACC, its: java.lang.Iterable[ACC]): Unit // OPTIONAL + + /** + * Called every time when an aggregation result should be materialized. + * The returned value could be either an early and incomplete result + * (periodically emitted as data arrive) or the final result of the + * aggregation. + * + * @param accumulator the accumulator which contains the current + * aggregated results + * @return the aggregation result + */ + def getValue(accumulator: ACC): T // MANDATORY + + h/** + * Resets the accumulator for this [[AggregateFunction]]. This function must be implemented for + * dataset grouping aggregate. + * + * @param accumulator the accumulator which needs to be reset + */ + def resetAccumulator(accumulator: ACC): Unit // OPTIONAL + + /** + * Returns true if this AggregateFunction can only be applied in an OVER window. + * + * @return true if the AggregateFunction requires an OVER window, false otherwise. + */ + def requiresOver: Boolean = false // PRE-DEFINED + + /** + * Returns the TypeInformation of the AggregateFunction's result. + * + * @return The TypeInformation of the AggregateFunction's result or null if the result type + * should be automatically inferred. + */ + def getResultType: TypeInformation[T] = null // PRE-DEFINED + + /** + * Returns the TypeInformation of the AggregateFunction's accumulator. + * + * @return The TypeInformation of the AggregateFunction's accumulator or null if the + * accumulator type should be automatically inferred. + */ + def getAccumulatorType: TypeInformation[ACC] = null // PRE-DEFINED +} +{% endhighlight %} +
+
+ + +The following example shows how to + +- define an `AggregateFunction` that calculates the weighted average on a given column, +- register the function in the `TableEnvironment`, and +- use the function in a query. + +To calculate an weighted average value, the accumulator needs to store the weighted sum and count of all the data that has been accumulated. In our example we define a class `WeightedAvgAccum` to be the accumulator. Accumulators are automatically backup-ed by Flink's checkpointing mechanism and restored in case of a failure to ensure exactly-once semantics. + +The `accumulate()` method of our `WeightedAvg` `AggregateFunction` has three inputs. The first one is the `WeightedAvgAccum` accumulator, the other two are user-defined inputs: input value `ivalue` and weight of the input `iweight`. Although the `retract()`, `merge()`, and `resetAccumulator()` methods are not mandatory for most aggregation types, we provide them below as examples. Please note that we used Java primitive types and defined `getResultType()` and `getAccumulatorType()` methods in the Scala example because Flink type extraction does not work very well for Scala types. + +
+
+{% highlight java %} +/** + * Accumulator for WeightedAvg. + */ +public static class WeightedAvgAccum { + public long sum = 0; + public int count = 0; +} + +/** + * Weighted Average user-defined aggregate function. + */ +public static class WeightedAvg extends AggregateFunction { + + @Override + public WeightedAvgAccum createAccumulator() { + return new WeightedAvgAccum(); + } + + @Override + public long getValue(WeightedAvgAccum acc) { + if (acc.count == 0) { + return null; + } else { + return acc.sum / acc.count; + } + } + + public void accumulate(WeightedAvgAccum acc, long iValue, int iWeight) { + acc.sum += iValue * iWeight; + acc.count += iWeight; + } + + public void retract(WeightedAvgAccum acc, long iValue, int iWeight) { + acc.sum -= iValue * iWeight; + acc.count -= iWeight; + } + + public void merge(WeightedAvgAccum acc, Iterable it) { + Iterator iter = it.iterator(); + while (iter.hasNext()) { + WeightedAvgAccum a = iter.next(); + acc.count += a.count; + acc.sum += a.sum; + } + } + + public void resetAccumulator(WeightedAvgAccum acc) { + acc.count = 0; + acc.sum = 0L; + } +} + +// register function +StreamTableEnvironment tEnv = ... +tEnv.registerFunction("wAvg", new WeightedAvg()); + +// use function +tEnv.sql("SELECT user, wAvg(points, level) AS avgPoints FROM userScores GROUP BY user"); + +{% endhighlight %} +
+ +
+{% highlight scala %} +import java.lang.{Long => JLong, Integer => JInteger} +import org.apache.flink.api.java.tuple.{Tuple1 => JTuple1} +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.table.functions.AggregateFunction + +/** + * Accumulator for WeightedAvg. + */ +class WeightedAvgAccum extends JTuple1[JLong, JInteger] { + sum = 0L + count = 0 +} + +/** + * Weighted Average user-defined aggregate function. + */ +class WeightedAvg extends AggregateFunction[JLong, CountAccumulator] { + + override def createAccumulator(): WeightedAvgAccum = { + new WeightedAvgAccum + } + + override def getValue(acc: WeightedAvgAccum): JLong = { + if (acc.count == 0) { + null + } else { + acc.sum / acc.count + } + } + + def accumulate(acc: WeightedAvgAccum, iValue: JLong, iWeight: JInteger): Unit = { + acc.sum += iValue * iWeight + acc.count += iWeight + } + + def retract(acc: WeightedAvgAccum, iValue: JLong, iWeight: JInteger): Unit = { + acc.sum -= iValue * iWeight + acc.count -= iWeight + } + + def merge(acc: WeightedAvgAccum, it: java.lang.Iterable[WeightedAvgAccum]): Unit = { + val iter = it.iterator() + while (iter.hasNext) { + val a = iter.next() + acc.count += a.count + acc.sum += a.sum + } + } + + def resetAccumulator(acc: WeightedAvgAccum): Unit = { + acc.count = 0 + acc.sum = 0L + } + + override def getAccumulatorType: TypeInformation[WeightedAvgAccum] = { + new TupleTypeInfo(classOf[WeightedAvgAccum], + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO) + } + + override def getResultType: TypeInformation[JLong] = + BasicTypeInfo.LONG_TYPE_INFO +} + +// register function +val tEnv: StreamTableEnvironment = ??? +tEnv.registerFunction("wAvg", new WeightedAvg()) + +// use function +tEnv.sql("SELECT user, wAvg(points, level) AS avgPoints FROM userScores GROUP BY user") + +{% endhighlight %} +
+
+ + +{% top %} + +Best Practices for Implementing UDFs +------------------------------------ + +The Table API and SQL code generation internally tries to work with primitive values as much as possible. A user-defined function can introduce much overhead through object creation, casting, and (un)boxing. Therefore, it is highly recommended to declare parameters and result types as primitive types instead of their boxed classes. `Types.DATE` and `Types.TIME` can also be represented as `int`. `Types.TIMESTAMP` can be represented as `long`. + +We recommended that user-defined functions should be written by Java instead of Scala as Scala types pose a challenge for Flink's type extractor. {% top %} diff --git a/docs/fig/udagg-mechanism.png b/docs/fig/udagg-mechanism.png new file mode 100644 index 0000000000000000000000000000000000000000..043196fc19cd636847043918317017bb9a076cfd GIT binary patch literal 201262 zcmeEubzGF$7dIf1A}9#bNK1_%9YcvoBZzc&cMdRM(V!qLDJjz3pmcYabPqk$Fz>Lt zuDidL_5JVteBRIc*W>ftd(J)Qp7T9%?+~CUFNuRif`x>Hgd_FjkrEOT#x*3Q>*yG0 zh&Oa%d@@K#ScT@|;)+t@;#7(bwkGCQ#z;s{0wSZ(pW%;_v_iVlh&03)mBefZ1fF0D z`S>{DMZQ%O!(_*MX64V4H>ZL3&Op6U0`IzNdK(jU>KNg5i2}B^)a1ON>!eHq4Ky`g zkQ!*u*|Mi2c%?G|?&;_V;R7QvUL#9+a3>0xO787_tRol0$B!i*?usL!FCyO>yXJh8 zN4yOSYY*M5D+C+{bw%bBn&D0yIfc%`E!+0yA66iVh%&bBa#WZzCOs|=VJ0v}@kz!k zd2WoASxI1Ag<*}iWH|VNw{vTdjHlDpWcd|Z^+!Q0B;POJ{BI*u)r)U*vDi?55>NSP zML)NDoe`A0*(Eo>qu3+vuCJkS%9$jhF1(%ObQ_*Q%+SARcbiV|jP;@D&YUTRGN0u| z@6z$?yC|kakRN^Ad)jvj5BNFyxN@YVpb-UBY*cokCpEDr4?oZzVa6NUya64M?ekK2 zJF>`Q8g{cwa>7}JBgpw%QxEh4>ZrIHbWyM&D16 zfz}O8T4&8RHG2IdvHKCv7#V}3=gVd17qlDK1brBGBmwjo*9aR3V?G9KmvtQv4}wT4 zGOb@J_ws=*e(LE_)@>y&-5_PO zWa4Qe>86voqgzO6G{|Nh-vTAni@CZEaIl|t_HgG8>|%SEHZx7VKHHG z(`4*El`QlsbR1E`KGM~P!3_j4H*>yJ72LN%FV++0CD1`*_`>|niWhh0X7rcH9rg;$ zIwW1cvpLb@>w>;WuYw=o;WXVX6rXx0LQ5hg@~90h4e7op(>sr}J8NP#G%wp^7V**i zQuJf)8V1v}pyR*!X7JpATdz#NgPkjeUY22*+XUlb=*gRO1C@fuYUnJaqychGd0&QTTmjxAWlD{D8}b6&6Rz-)=*7esR4q2u}_V&jFL})yG%dVt{YN`r(Y?L-+Xct_29r zlg!_o*Ee9^rMG{@D-}++_U^-zlr*COdZqi>iB16rZx5nm=`F*z+GW~29d1^>_k7{` zTJX`uqjw+gDox#ICtx=gSi(q8a0uIrl1^V&($4N5l3jPPU_GQhWIy~!AlLCIkhM8t zf>Gt(Fzqn&p>!N$E=_m14qF1NHno@ZX~uyJacXd0aPj^K37^L zTg7d1kAEyuHI`i3aj0@gbP%i*tYoSTa`?*mDnr6C)80JZ+|NAPjK+yfJFF*zVGw(* zb#3~<-kr~#`NZyIc<;{M$|5>3hIn+NNg9XC19^^Z4j^+DQvt#Joie5&<^v{AMXlu3 zPj~axlE;$AGLO{*vKO=8rbgd)$y)Iiaz)q_{Td`nn@x>Wa zL12n>L5D^pQzR)9{wIr1GoQqUIfjEi0fz5dyEd6rQ>tuE@I6 z2gP=<7`t$^ay{ep$+k^)qjl>n!Er(cj?`{0`(8rbOl zQu9qgN~wOh{_AgYG-GWo{!yh5j6I86i7St*FMLokr}0c}sxVE%L*1sts?gKg*yxU} zbAv@goJsANN077Mp(?2+={vdF?wH&%>(5)jNpFEF?YGn1)yzj~AfB_3BhxKmzA;`S z{tr&agZ*Rd&AV9(tJJj;yFoVrBrXF^^StB^Bu*y$t^Dv+!PV4nU+lA;+~+?nbvWj_ zc(rio(G`al>Qu&7227Dm11kh7@?CAAB7!wSSb|kTOM+hoJB27g<{oq&!|ovPx3^888u+-Kf9K_MajLL_Ie@!PT&Wt$2Yj+#Rprz=5~0JppvgBi6NWzVPQ#i*ID zQeX8y%50hrFqC4GqLTugY@a(^Y#1dizHFydLQxJ)lTCk|#+`O+z+kWkV-fZKiyo)a zgW)wHF&h0kJG=C+9YV~l65grnHy91oAIs2_a|!S?oz6p=#MOJ$@2FF%zRu?_)o>}* zI}G7^V_BBx9<>z-8`S91*q#V=CfVT5CDrKF0GnMaxo0EZ`ZzblomA4H#?gZN9e29b zDc2}VIU{s>*}ps2{S(6NzRtRXRigosfDXv)e8tinGzM}q>uOs%rBQaMiJU*Dk)p0I z^)1?Z%6ioM0X*fvFMA7fvf6cQleYlBT)AL>n=M+KOQJ|(T;gjr5yXbcgpi1!ezMr0 zcldTr_)Cx!s2PMV1C!D0DCu;G^183U1j(_>F&oP)vfZe|J##+;;Z{&&31@i}?98oD zo(SJ0+XD_g%hps^dY#kp$-{&9+-_9CMR@o*vntVZun;vbhZ?@tpt_}|Cg|DHYAAn& zycUlO%ko(60!6FlddLPY*8>tP1t0u*{9bS<>||T2K7u@RQ+_yUgs(RL-H2KrDp#&K ziLJZ^g;jvX8s7pa<$UQK)Xi>Xj8c+L&ItaYZE<+w{iz2X&|3xoMlnv z1ih1vy!XT#CDX&Wk~V?PUZKb5Ru02N=g|ZCXu6v9Cte`lNNz5=MStFh&yhs&<)0wU z;jH6lJw@uXK^6&*J8XN1+0(vwO=y*wbTtazE0Zi{ zA68bnld!fY1tl@V)I1+Y(u^$DS=VuP(U3g1lH%g565~!aLbyScK1a~v8BcyAML6X2 z!<~7^>8xgjATCz(G}b5fI&tIa9#m33@fEtKJ zMMWj(U}VCt^yu*)(-Hp@qBe7Kvf~E;TwPt+TpzO8I+z03`S|z%4>$lE9IS{Dtd8zB zPWo=FHjXsE6ZuZ(k+GwpgSnlPxvdS=C0%_3TW2RBYU)cyfByS@oyKnFKRDSq{vj4Z zK)~e}06W_Qz@M}cQw1;I@++FV8C$76GPgFiaYXO{ay;N-7yNC)KR*3n`Io8cKc=#C z@bdh1>R&$nF;x(7DZ*bw`dweYy+tSth$RU4Q~5wFb@LrDBqR|esYjyE+>ke?(PL;O zYLRx}n>s)?)47{Bc{!-VagrJN7z6dhZrpu?_j2h! zU--`#e(4Jy!=Hq}AX>`(=;2RJOSjvi8ju$p3*oP{96w$hLp)iuBXf_;aeOr}$%;(b zj!3H4W=^NQPSlF?o^w5A^R>bxD^*}?37c}VHO?{C5dT>f?z>H+SghxxNy*ftt>8+D z!F5lu7iEQ#G`A6n=b{)dtUG^YBufA>%1(dQ3Y@vC8__-J^oG) zk4mxDN6%+Hcaiqr8x+VW&i4?k*{Y=Ie#!d~lSwF*9a>K8}97hcBrjYk$b z@~q=wpUnD`pFuJ*N+})!R6f3h`9$NB|X62ax21#`RYZ+6qnMfn)#P+uIV-J?iUJ~{AiU*%omtK?nw(c0wZ zCwh4%`B{T>_jprGkUJBv9$bKj{rq{|n~$h2uSEE7;?A(9+H%m3+V~%jmWWNAwk^tnffH|bO@-u(F_v**2HF`euk*vwkJ7#jI8^ad^<1GzyvU zGv}5sV&_K}p66Tb{s)+=Wo!P^H1cv2;07BUb}PN6J+4~{y$Tb+ay(Z-CI9=qB|?T9 zlLChUyx)?w$oTFPfr2}m`_f?MUWD}4iLqLM+`&O?tldn8@FmHzMwbn`&VFrFdAaTC zmGhN45H~Rt3XD8}F2q~%a1dK&(k*PqEY^`mOu525Y^#J5rCQn&+FiTyA%LbYM&~xH zDpddkPV8n7EzeiHBsF-#fyT2)LlLXv-{~D2X50>aDzm-{)$+J_yI8$j0dr)xcWQd| zv@+(c=PZs?9ppLvXi{YG{sE-hHoX2`&?hq)8Luf}*4y6rKesd1>khAJhf`4~>+9}s zdsGq9K|ga0V!~#pJBbEWoO&j&^x<0*^TBGe<(V4^ph02-0JJPr|sPifc@6-2& zR(wl6GM1I2rh4zn zOjbE9;wWF3iD;!R?H-d5m!ME=-`SYLKD({N(&G$;?K7hDtk7=Q6&6!+Hx7uHMRSvzM zdZBtzmCfjusx=b@^9v3;-eD%q;fSft!b_5(H_3cPjMuO9xc0nmCr#^v$5@7r``6L9 zGX6K~>Drn(rrEo3XkR{fxGV2xn^GnQmDhZ}=TQzHob_1ablVK4rQmv#{7^Z0(cPSSB6Db0Wc0LzKjoTNuyLmqhH zqL6q{%TGQ?!%TFRZZyB!-D<5#CF0Ti@u1(c? z&3A0g%x=CIvwYU%Iy%x~rMe!8cls{f%HEPeRm z_h|ADEL4>7<$%A^>14WJ9fh)Hw5&QiaQABjLuig1z)OBj|2?pj_0}qjFLQXnnK<;b z_uF**2wV9XDn+&}4x>RMQ00Q=r0|2-8c*Flf|Ifg<_H(EBFV!QIcc47aDRK{@m@S> zysoj1wY#mGz4$(=W%@ZKpAeC0ZF0@iO2@&mR8B@3FO}-|+P)=8`JUS1WRx zguG{MTKf@6@zCQ($snvv;kxrj`jsS<34)En-eyHNkBg4@PNC-Yfa(2bMN`c+&9Bu8 z1jnBcst_l901p9i=pYe>5$(;_w#@@2m!s;QXV;!NCzCS>+fFiU4l-2f+MBWERp%UZ zJU9ypacdXd%8y^Z*b?5~)hm)edtns&TB==>P{+lN?Pg5h3_Q2paslGWL@AdZ#5nAE zw*G|5Yp?cMy=^Y$+HDF`il`vlDIHkDm_~AM@kUPl^T|{4LeR4tW>2D#W6DeLXwX-v zaZJWI(xFeY7ccIo8&Li{9~ja|?h>{ftL^tdj zF}Hfh-?HGs3zImTy&iIp&K0`9KGvRhXg4=k6xroln|-Mq!C@o4PHn6^*RTG%Vmtq2 zc`mvZ;W8E1!Y_nByUCU?l7XLyd!F^my$j8@mRmEc+9+Jq(RylDmk%B1B2ws@9Lcfj zn2nh#f&jCf$W`-$j?V${^GSo$-1NxE<`Z6SO&hu~K?@iAXdi|xWx}kXz4QS16^6D= zhp5j$hmvxZ`+G*=oLz^b(hW9Q`AJDZo>F+t6md{M!;Xf{=j5iEtIFm^q-k`Y*99ZpIv*clt%~G8b zQUwjT7w^Mx>x&Af(_N(KaRtIJIQb}(YP0*ayge?47OhdkddvfDYiqrTHFe;QJeFm7HsZh@c@*l240gH~L<53r8!#E6U06J5U&0rObyOzlynek*j0~j5iB46r zsQ^y$^w;xY6)HMC$uP3=WF@|~1`E$XH%%9Qt(Mf4n&kfMtT38w(0;1!7TC>5`N9Go zkGAZ&Dk-;nkQ((+TA`(!Y$S+Fh))#%XSo?8`>(>%VV$>LN{8(qOleZgaH^|!4|aw{s$=O*?d*-(X6sJQ zg29}WNwd^=$xFu(?eRD18S+;E(RDjH^9UlI!nM_SC$}@x^DQaEWjo5`0KfvLgnWQu zOk%yKb(5DuDZoe;ePXr@SXN!$V2!)HkW+4R6LqYpY*sc3MeSlU3d9Wt8X2tFm+?vN zW;(sNNTswQ#!&REq0t0L&wO~wG<^Z_tQPTM-4}X`Do_LQJE?G;hyw9=xxhhF=b}ZY=nJ_VqhkD`rPh2|X>oJ{tMfkr}ERiBl5S}RR zb-P0MmDtQ3yiIui;_Og=T(8;f;^r|oCWb#sO5Kdx& zB|I>&TMIaja-6)lXz7+?r*Y>1OQt&o_;^Lop-HFf&S=)jrSI8R_QHzO-}GJ)wi8fg z$DJ}~vh1}VN5y1GWjbxg>r@)74aTP{b#8o=AukSVIKuj@(4yTrfRvth*!4*XM1!b! zgoxGnT_5rL@>+>B?fBlJ2^V>xf6cK!-?a#g{#wa8mptfsRNnXmEgnLW@DDuRMrUNO zvy$0z$AG144pP+s!R(^pnd1aW-eom)qpl`>I$6=JDkdTlb_C6|&!>QT$@bSjde?+9_J-#=)dYRpFhgr3w5j z3B5Wf(@&F6wg{~0SD0%HsT11jj~BH**gX-irh2hGR!9Cc6SH0UT!6n@x9ns7s>dO1 zL5b`Gk?iD%JS$a_MP(*d&FNhTA5`J~)3cv5eq@DL(yB@^fNKDlA&kho(9AKNxy?)BjY4%=vE#9Qckg3$^ zap|EG{24RdRS{B5k7~q@Kl62#%>-sTNtJ1OxTvqD3$nf)OxCc>B_Yr?`cI$4oq#iJjD^sSv z7IE0*$FRWykP4MSEn?Y8y*3egA(spD;JD|#aBYi$9mIovt%NAM!H3 z*S%UY+i6~O3j$X@`+IZadylB<*S%;H;fuQX$MsH9V% z+60on?`0>MpFSlQ^Ni*t3i_r})l=n_=sJdhsSxrc0TyjF9B7ZGM%50Fps;ud%k(2q z`O{?8y_E~>74DX_{&WS|dVJ~YBWabHLdvHf@}{cwq}D*E!88s}*93AD_LhCh8Ds)J zc-x(UzQ6_}g|b+FuCp;h-+|(aNr&t14{7W+UZyih9ZBn)ocl&hDEQQHVBS6+vRnsu zA9Oh-zj0F2Q;#)%(1OG4)s9?C!zNIDkkl)UADEv1{HzuX7IZhNdgO7^k7KVFD-uvP zE_ox^O`qeIR@j#9B;jP_^?}iK@NP)RR!d9T(_55YDr=BeD(wyg<-!`L0vQhG0!OP! zl=rp$rXMfO7ZsDF>T;~jatqmdHVxZa`#b84;Q`ooN3KB?G}$MwfMRV_>GNV&*X%eC zr_u)N_D9_U3c+_{YB*}G`Ip>r?1zB2pbG7+#Tzs?$7zTbB+GE|%s`3OD2G%#&te&( z!h{O$$-DDYx(Yt`FNLcsctREfR&i@n@3~MWB<#Y^+HaYX>G;>Wve_s@XlK6V*xCfo zonoc8+{I=AS+y{&`LwLXEhfIfJj0Wf?o{Xq8?n4-=`y_^d#Gn#s!8)Wjao0Ml6a?(Za>_sWzI1m@u3ha>si zyQUFpv;iHp>1pNZ0l=O?FDv&oKU)jiQ`c12HLGr7G&)ffrQTalu6-Ryk~SCLn-3t} zW`#ZSPT%KWP*YZlhD^V^O#cq3DEJuXW3DlI_C}h&Im>+L(aH7muB9I6I@5U&^hjso zYmJH;JYBf<*=R6Phiu&YsyAM!7OXpwMjSO$Z{-;3aitS|Y)LT+ z=EpTgJz68-W9R;})1wza8%?ov5g$&-8+eUq%9eXs@x8b58Ma>U3Ca%zz-_I2iy^nC z>86j&IP3e9X%5{@B-V155JDqXu|vv`z;zj+bE5kh5>~dA+UB2p-gD;8sZ% zb#nXZ*^Z=d7vmHDb|0<)9}PsjPz`dmcVFXQDJCjU@^^QTKl5T$(X0hhCorBY)bGCo zUe{pCm4AWeeID)USm%VZdtlyfIw^ZIedCD?YME*>jbTN4*L}yyu*Um$S}qQu1oO3{ z+|7$vjSqsJ(_5y)=_!-WRqib_EO>WUMGbJn5-SV5V%OeFpe#*r>t>J;OklB8 zyf4I658GX`q$Z4VFREP90DB0|hs-{qw}Zb6Q)#c&mXnTFJ8swAx7Tr)DY{Y1ni5wA zZ4!jahq-mC$k{{Q9~pjbHuP|yOoI;0CRvOtpSnsyf>`rM-dU!J3J6!_;6YAol+$MY zNIN}8)0fg_ZH14Ofivgbw2&{XGB)sW*x*bP#m@mSD?tQt>sEsRzPoW*|5!eC=w)$y zQtvO?t^kbPdBOlurNh4%P80Q_37vUpaL~7SXRTdhP9`E#%#^~ z0<{NS{xjX!_xef&B;Uv*R?ZC_cPEIf1Gn6Jx#Zr}74ch|gIadxj5agxd+j%W;*NmE zhnB!0uk>$KhXvjh79_eBel{0CTQHY0zSZiS|H|vw#~)Z5Zkf3NJjs5~KM<-_M0@7% zdkY3)$p5CG^SFSq5|>#{r|kVKAbYSN&BmO3l(vySD1srroNH@{!QhL(obZC=VqAXY z%q(+w!;7`{*%u5Eg;DY?CS2NXN^J5%R)e*NZ}PdCYjRPQjhglK1|U4z;pgnO5d#6c zqqH|!&8O!4R}?y$R~Mmg7SD{II-EDs#+aOjN0AM~w%fe=k2)qDLlEULJw-iGGuZu%G_@jK|23{NInra{Mo zX?^a&0$N?Pg4512tpVfoKhJdX2zH)4SG8ac`uW;_B!*7Dh=9)~@*MT&O^)w*0S4Zx z^kuf1oqI)8{?W|syN)=etU^-1^>dK_vEF3n>j=0{FDv@7njh?b&=c_yA-YUhQH&&h zKhyBIJ@z(?@qN}#6;Aq5RGS&mRZCZN^wr_Y_r>~y>a?^Tti>O@ZkoUk@-8Uhl))ES z>PExB@GV#1{kzGU&hI;Z&K}7Z3n3BH9PgvwntC3e8s3{yX9ybZ)n^{nyIfIH!?4xp zlP}H`v41JtJH!T1-z5E|wk{Cb8Uj;VAYUc!cg-^1Mw}Wo>HmH}bcINlG=}OCoKGR* ztbf(@PwhnlA%7sR)xXgC?UU~*5rQ+^E9?3%g?WNN`dfZDKL@xU>rsSXo_N>emj0ui z`d!%XV+jzahK%zCqQ4rru8ZIdgQgJvQW$E4y-SdB|8oCem#6n|;1J3$g`q{Xu>8NG z{c8WRzzEKVvwj-?E8{);>Xzpwr9@?ORNoiSd9mQ*3_lHH*iF&5oH ziP!hvh_qaHp`c*5QOm204C`iUiaR8>Em*#_O6`mJ(ShA!C*uo!FJsebi1xDr0}LZA zy|BW{yW9^JLWX``st>6pn=~SE*_6-!JeT^hgCZu^S@7SE=M6vjg=$x`5oY3H(QC=` z3*8z$N9?{y_4BXaHFJfbCTAjYrqyhr{OKzOd|%{Sb;Mr9ZfW4q{QGA4CNd$mAoHl^ z3cUE!?GQ2}F8dgFj#vddlkSx==~SX}&^^e~&A;2{s%+I8u0m=k9sy!jCjZRNg@0Et zS@i%?yEfxt(tO%RWbypYb^NyX4F$fn9~JVoGA7d4f4clt8x$^VG_~ry`y;f%w~J=o7YvC4g2Qq*>@0|wiKk$R25-$P%D-;(BQ%aC_ruwi`XY)jsr9HtNM~EUU|Wn znL@+q;fCQzmpQv-OGym>;hNiNHiN-MxL8Zsg?rh+L~>2q>?z9v#Xv zIPnB}pDsX}((DA^%a5v%nf|f9m~0pG8@{0!oJ;ZZ8M+sOz{A2I!==4+&Dy3`wrc7b&x`7^IE6Rc z3=bR@+bWi#G^=!*hy4H0oT4Pks1C^b1Nh>^`I({0U$#aMgi2DjVzgqSFqh3h8C0xm zFXnk34!wgoXl7+4z#6wKAT)lQ{1!lw^qMv+T_E!S2W>6wqHIdFGYSwugFC}~H0$+f zv$k{Ycz!8qyGQOB4@v^tPW~Ar;GqIPJ)1W~GY1Rx3G+dMt#|g|Mmo~>t^S|T)<=$7 zL!2ydAjXsR6`||-IK;K6y5mZjYEkQFz9MDUZR!t~-CE;&q)?Fex)qM$^X1^cYe|l6 z&q-@7nmg1=>~t*a?kfOiPRG$vgyiPxK#)L39dFNS6Jcl{O}MdGXlP;eCfQoNO%#g` znS$WOlk>wP&4{gxv5Gun|lvv)CUD|P@fF8;E)>3O!28)zvdLwk%Q`ML-oM)sEZd;n$P&D z-5Z^PO!u^fs}Kr2BigbZfAVr>8`hfe?jq`^AqY_`DBu;yiYTluVOgKgaH%%u$h%8D zrE0k6M@e1}TM(XC$#)m;kFC_oGn@Q95nHwJb#f0WPjGeltX*7Hy81QDL;rk{h$kAc zwueV+gFHHJoA&loyA0)tE7$&K55A*LTagdXHKR(`9J9$%Rd6lWXoINud{JjTS!VtG zaQ|C;h5@XF-{v}z`U;<4WVp@ux91+ZHx>dya_U{ZZmbFyI49x$U5!7GP2`oH4+Rj5 zgoIn}Gzp%E+fqFXw!c9kSmGCC2E1I6lc_HpcEl6(2|Al}{8_|@`qov|gkA(9<);AN zuG?9%ZH}>DFqo%^@ThpTjL*pW7O-eIp{&pXgHDq^G?fi9CF}|FR8|g7dcf|IPypKG z_ZAaesq78%hMh*&HGq$Rr)~9yOC6EjnuD1@mG{tqR?2`z@p20PV!Otg)O|ua=gO7s zdZH~E9(r15((-+_t)uekLXX3aWv(Gg&$+rZZEOSaqNr2g8RgxxO^}G}1gQS*Ll-SI zinQFgEe$ML?^Rv)lg2ZRoVRSa1&|B$Qk%2y`u;~iv?9d2Q81n=>CL}>z4f5Y#gfxE z7n4Q1jMsY?EF5YJUSKXX>VFY9-nv<2>AAeGd%v3S&=H8+4YC&O;mduN@R3p_CW7q20YI@P@s@i7@ymJjN zC&`OnBChy(!zW3D0=+sXXSGYLudKbsSoHLdy9t4aOLocXbGp!3{8lm-5zmVV%5e+C zHIpwd_=*>-Z3UTuPT#K6T_63dqv8NI-O{S`R3DcC=p4w_EXb|U`XUk0yLX_Yq!Gmk zIcj{cKi^-dqSV2EEw)r+jP2ypaFf7ot`1D7y*0KV&p1(bWldy7xMSk;=VIb@YpF+Q z^ZRV2a6&@t=d%G}(+t~@EI0+4pLI$(G8DY{y&V^D3LQ=cqO9z{9tsCP?DfzlJ3J>U z-IfvFEZVTK?{mZrS{qEXnW^^}4qQ%0)8IJVo)$R88Ot^<-h|+gjVAcv6(UyA6e@ZU8AUoJvKM5x2yuS9(SaB*P}@PLH-mtG2H4QYu9jMBX{}l@Q@G z1_;dT#=?hAFwD62`z@rmCXro3ziGj+uM3B@A1_J*LUm{DyfzADB2*#pi_q$iDf;_C zs;38Q_&gUEfL!+rX!UGxulIcT$W}ghrHc0=ZBV1U(BC*1)As!>OL?CJK;j z;57aAEyIWgUsgzal;yFU4^q&QR@nEy1@KekDnclPstb4^HeRh)pGwm6f$(P*KzFCs@I@%j*a!PmWFF%3*#Ys@Ppg z4>Pmc0zjL0)+X7!U0}HL=%YI1IjU>`{<@{d`HNem(b^ZKyc=A$i~`H$U66-A6#@gV z?S;sidNNCiEGZ%=^ojb?ARwF!ytj7e8nOnPkXhebUPvxci4`|Iktg46plDSOha2m) znCFEuB`ed(JTBNGCn7a7@uI^>{Z&Bjinil$w4q8-lkcNsLX;fUd?uzNT%K710MG$m zvCbT=dIR-|!bF9>xaZ;*?D=y?UGn{vZq};Mt9lu4RS-3d`j+|@wMcR&DgsJ1pQ<~q z^sr6j9kv{-DYObhEZRnmHg zar1gUudQF$JZ6wV{;=EOIL~Sew*BDTQ&~9ci_W0Zc}E2TD4)Fzac?E1!Fl`cssK51 zy`vt^zri{RM1+n_D&KcBl6SPtcT*)KCQD~x5^AKqI-U9&dnX-==T{0B0;2_D1VApE z#Sq^+z4mWKOff!dR2EGepB=0bK~5XM&BUgO${Y>GYemhw?YO!G^{~TU&0ep$B)EV5 z)Ph&)F$(~L?@OM{Sn$?11?#BrO*%cpN6_9f8Wak=ROU$KG7-a0HJgw$1y zIKe%ISUHnW`(h!wCf^LC$xX->Q_niSL|5g1f7W*40)#nqlW_ z*$OtAFy_S>2$BryvpM@L{Of2p$P}FpO-F?^OSH`m^J}3&nE24Z*ug}L?lQd@t$!1n z@KWqT)zSbY->utY753tuJ3=e1to0|x0UL`3(Yw{ntL%o<4mjPdM8d^X?#d#&gHSs8 z&*OP-wDPhxqZ8dcg2{RE%yhX(LGlv}p#m3+7DXN6dVbh9R&~j46j+=DS~D4b*~Ulc zV7WV%{A`oC!Z;q!_=+ZV)n%hnbvnHW0Y7DzNDlP;Rvn0f{3hfsIm8o!zEJY8Cy?E+ zRU)r@@zlEfoO|YE*^yf{w|lo6-D2N-zDY{rdj0a@O6(BecHvW86q?5&Y@e1(vg4EX z0Q`z3hMTK}5|RkT>Rug?jpPswp%IT~APl@QHeWwVk7P6JT~55Y3{5zp0HkKaMY}wT zAuNfvFzfhae}&L`l1dR5rCMR?V^$m@C$>XWN@mBsr2K(5qsC$$x<2op!K}VBOC*7A z{d2oR^1Xp!<ir2;n8al{Isry|efFlRRnt#&N2r zqY05hAR&2)XmUK(4KiLv1bn!ofEV82use6)tBf9f^P^zu5b!Ra4l28n<7Y%A*`^U@ zfs1JL(AhzzT?rGwbH2UzCuVMudYvAzR&@n3jRGOlnF{^xGErc%>|n+n>*&cb z%Yi!k2GL$?p&CR6SFxtbQ-a6?lXXz#h4+UgRBBJ-bpu04c?ho)?&3n#1yu`iFztYj` zO?@OCN)8?{&pqSewhnHkaA&OB8JAhetDotws-M8cFH?gG3a#u)(zizEAIpA#_dr{c zMa2zH0@HU_CT)XYxV((?TeZ6V#<)WBSruE$y5-x|$6mH!`QS52_Iv7ZK`*h>3q)#7 zCqvlWo>`O1rEvOfhbD=VA>pEC?_VcER-7_a`L?lp3RyE9bgS@5;Q~)2Oju?Qwz8)4H zjhimiQl2Yr@|h>C4rbbZ)-31F5ZqPsgvaH~{}v!-4!&jr!Rd4;fsd{@)^hjwMh zNJ{;!w5VR)Dz=f&mg{Lb)mEh_lxKZs3#XJi%GO8WCo{V8H_c^ z6$&{L=7(IiZ*ycrq!UXz=h_$))?H~6PS!`8BX!Sg_OnmKfHN*L;|TqE>pEq7MFgC@ z#N`tlFh;exI0OVHJ_634w6-WK04dfspo0hPyON^L5sxQZ!lG;l{ z7BXd6$*?6bizfjc28!Pl;&5O0FzQMYU@+;4q3DceW9o1j;ah20(lyszNtGp@yvvtn zdt-ZcZ0Zv+c2Y=&r(7M!!;C7gZYfAVCk8zZ3S%TaeWSo&qDSgb3NJL5sEGjxd;MfK z>fZ2kz0Cq|_T+r;ENcK`!tb4x)qwAfl?>xz3a^8HMC)iC=^)W8wfl$57O~9f>K9K{ ztCU!umSlcvQ{n5J8h^-aYWE&-78op@XazCTC^RtM$X*-V)jb|4H<)6>*MO!|4rD^% zkSe%!qR-S`eB zh2;fHkF(*NwlY*Ar?1COW;an=ay7 zDd#>rKpZG2+}&$M^(B_48o4-B0k|3E?4}D>5qV1 zo+2k%KR0IGrp>>R(!SA3hf+#du^KKDpA-stB6L%DeKmD(AOJQA@uUJK%90&zPiL;G zb+(B13bA;5euXS}5qZ%Ls78koKP+2Pyg|vHdgxe*TfNDD@4};(raONd5NV1C?ya5j zyy-(J$1Sej1WCu6L=h_d6L3shGppe8a`*U#D57yhWwIO4ndhH0I_9e?yPe)J_Tb9tkuh!X?#O%>~Pdz2{5)fzU1S(FF1+fy+QEg!RZ znt0zJ29K`dKY%3cJ`~dvmUm%FUo zf=y}0^LbJiXrpM(Xf8-X!**&*wG-v(`=|?+YKMz@ZGR3~V}+OJ)lXK~95rS2w#Q5= zNqiHT=VsO`z4Dh5nETF|$Y(VZf;QK$(q6+&{4DhcA`>XMYTE_gPAL>@eJo>I#+Bk8{SqEEBJdfU{_4?; z?8xDbz5e&Ewui@}$cT{#kiomPq&(Ud!^Lv%X78yw-$Fq}JF*ULSst}6_!CtB_Tbxd zd#AS`fo`MfDk;8)&g9343dRC8U+w40`a!i%{qi72X2kiI$Lt}OaR5W7@RzKMFE8ju zect$aHuHUjK1jYKh@vy=R*k_gS({#-bF!$9{^fw}pTKbWhU>CfK;zc@OV&Mzfqm*l zc``qz^Fv09%lkNwcLIOOI^puSXT?U^e>qkCdB7aHyw(&lcjuR^`DG9TP1~#3eop5{ z^C0W;b{I(`^B)PukInsG9CeC94D1Pa*8Y`MDAFMMkz6lSS$@TOl?5^I|5JnXg_}Zv z4Qccfygu;CkI)&=JBI`6=8F`iM;=eD2%h2<80P!D&=$4wr)E@BYzp-W#K@wO1fVz^ z%!QR65E+dW;y6Vp9npj026oO==>x?4ESU-F8Rm6O7m+D!O zRrJ3O`!N~sp6};d698A4-(DeOZI0F9PzI9*!Oo6S^isrDhpPb{?^#VJq^X?cNcHHi zWc&w35m{^m{|p660R?2Nrk4%l#isVhefo>t(L@0O0a1ROE6{PbZzuYRdqqnf3uQW9 zf*Ui~zC(vIp`HIG)^F5z?=|T89P@^W@?klb?^1G%RTwOBSWQ;l0YNuKUrR?mt$i%_ zELY=>l$7(p_T!r!J!2TxztGxkPw}xx$+yhZf(DLam|fYPeOALv?|hHrfqS$)9-@-- ze7srf) z@p}EOs?xEt?!#IY`4>i?p5?xgj{KA?`t=Rt^BT8PG_BODw+IFlv zya4(~9q=7tZ+NasWXZYJc+tY}Q$xGK?s5x=&ItDxF|1eWbK{wb4VL*ebLFF;Z4h4X zO-f%D^`hfQV_L)I-ogId{X<396Hw+6xq#t(T5oOfjn0p{U^-Sj5#vN5N3@T#ZXu0G zX!r6I8yJ&k42w;=8khk>VJG{`B-mafxABV&vTAzIX8I{B9hUa)FERbw7E{$-zde7p zQ;kgiH*p15BlHgbZe{SEez?WMyFc7GVbS8&bCy`^x zkxVEqE@cZ)SCOq>O&G0Zp|*l`o$)Q(&YlclE?WKJz(XS`FV#|uX+n##u*CAIH)#{j zHQ&A1o;pBPn#z>08 z@F(K!s;UieSJtTD<>+IfZE>NCLoZqSvTmV}Awa%ax<*UPfs*Dk69|WSHn&{krC4FD3qlBF9r7lp~bppgVq8749dI zqkU=Wj-5CfS`Nf@aYsTVL%I4{`<-1$#5eTt0&*i^XHdhwWYNN5^P{Lg0MoaE$Zd1C z_32}qPmJnCMpKl`xNrZ4rT#+5Kzgd99`!zt^F`%e16Ygos_P2Upa>F#JiGcnpnhH& z!oqTXrtpE!YvOI|9jr&y4bEHqVnSG76Iy~jt})1yx#pvtpEiy`1t+uS;#{`)G~b}V z!Y05dYS;(V1?#SK4Wgk`xMrK2S6J@UZV#7?BD$KSBb8!*VC*Hd_;z6l!lxX~rmLJy z;=djJP3K70#i$!%k;T4{kcLFDs7%;P78|v{kk_tSfR5L@m+^s)J`4J}(=e&z&9^V0 zTu;?b7HLGOkTdymY&SJll9!P|(Gr~G-OMCJhZeVMX-V2Xe>F`Bmz zP4j$6Y{rsPQZBw+2NRD}-q!p-?7d}JlvT_NPx;Q?EI84{YO^ZZN1`V z9O;ye_Ts);*OiEdRi>J>PE&xr8*zL=|D@6`XzAL-r@i)Q>$E?M=QxPKzK_389L;iB z>Jl8Y^Y|-P!aXTFfa6LtF@}Hq{mL;xIGEpC6p4_xlSRGmLNdNR{#NO;6#~A0^TqU6 znrrueh#JeTJnb%|IdPnk%zgPIB|Dm3&3pt^;+s51^?En+`vF@XSAGX;Z%{z)h;gm zG-6I`vk%mEB#kXfXw#o6I3n&oS>wP#G}zU#5<&w8jX1P@OzVKa$h;~hb^w49XJ*|e zuu*u*8b|%RKK_ep{^Q5DHrFECpCH~V>rIhrA9&vAO?=|(7KEqwTru4>&LMa!jOGw$ ziY;#(;!wVQU#oKhq zE;;K!7mVmJa8j(@YMV7A6 zpG52hoQMZMS<;LACi{nWKj@s*>JcG7BKX_mlNWTD1bTl5(161+7h{*V9S-i!2eJuV zv~N434>%wCY!+{uUq_hds`JZ6KF_+S(!A?rJT|0WATu&;W^8|ElD5;l2h4)9)tjgA z{Bo)UD1scAM6OpY$r=9=YWKK))J!R}&{s$o_BRcQz z5DZLQz0bzx02WxCRJ?i-b`5vJK$y;%J#J5P92z;~?s$a%W_kY>L&;!SH(P3{Gn%wDINYgZF9s~g9U)>QN<=L_K5(TN%|<)sr`Lqub4`|UZHu0%HVf!? z;o&JXtEPNa{OYAR&YzNN#0KiOs7Bj5l>7*NpEoh%Qeq?-elRYenu@6AQyUV#q8LKx0 zRBfYrW(x-4M{gLWs(+$Ik<228r8xyeSUVa{w}Y{00narxa)>!|K;+Hy7D+wgAlq$wO(~`u#d$W$JxI z^39I7W-O2@R#24+>|?P_Eue_5Cl_`+zSZksw(bpMnGLt|k8ImeAKA{Z)%# zk`Hu1T&KYM+-w$rNjgioF||0idTSG2J;cPnVFs6=c$;4t46di9N+q2a^fuNBii}of9S|S054WHm;0CHHjdfzLNmOG%y z1!+OM08dnGOCwc5TXWW{M^J+ewC=v6mzcJLv^;pDk*UjDTU;Wa)^xG6DQ1>w27ngi z#ipOb{mizHPs=oy4v1?X{nG*caaqin;QcFbEMLrhxC~@nkl9C+Ilw>NT6LcGG)0OU z6H&2((H)10dxY6;$<~^@7pl?n0Md#B30>~2 zR&d@^fV$#_9jVrbGJl;dgh$BfzON)IE|_;pXu0w39Ypl>X7#$`8-iwcWbKCsGfZsu{MC6XJcqmire%YIgjVJUY&px(l6oz6&>W}P^&!tp9Rbdg&7SNx(^K|Fv6*7vJ9zFkXe%P5jnHdD~SUY%X9*%V{H*LukzFX2w zNe+9u@I1YL6JmGDAS&2W9Mjxokr57OxM8Ptt%(gKb!J`aF#FfpB zHwqnm*gQMGI(~YZZGG^_;UT|q-b*^{{Pb{ri>FDGxfn~Exfv*&5x3v-8fp(dR7|4| zINch~Nz0I))FZgX7pTq_0O03?@QVpV-#0>*w9upCiU5j^(&m^eyLUd@OwY&tftr@M z?@8(HaPHq=83s0(GTz~)rE$J?vnu+Pz`hiM=b^k}ZR<2&qhQwrbDfFQie*(x02M}B zBlmq0le^TAE!zO)OMlPll|E>_hmfYMTl~Jolh_1`Z;>PR#6dV9y>8~qy9tz&ip(%8 zL?5WYD)wE-qND0ZyU<;mSP@LFw{K)|+#rQ2*Re4WEZ328<&o#nJ41t3`M6t;Ig(A<>tTxldFn(=e~?sR@WtR5^~tiha$#n-r@glIKrw9PVO6#h+N zKm_8H8pB1~QBw8pc^*;4BTe?z)13kKY3?%aIsQvZV@vsqA5=2s3~;s|Mv`{KTK3&V(s^oGNRV&CI>gDj%z)Bz?#=JTjDAS3oIJm@P=L1G3rvfnnzbIeC=mt3LpWG{K09(u3&GB#V~0{b3S* zkwXFhSerqUxw4!7=$}5VDc>=1=YqZ$mYwcSWUlnai*1vBxSIFtf4MP{DNM;{+P9MH z^QeYQ9b6e@d&QPE2R#b^ejd*si6=A}dukj@(p#aCAC9u=i6zmX2yU$nWfoQqT$-Y-KZ(eK( z(9fQ2*MDA_V`R}<_sY@_F#F*7^VUeQR=?|`9JQ41MKV6~tAm-T(tH}srdUUt`-tdl z+se>iOFN$QzZnhWE>>*Oz;>nlL%-VSp)BRYN?MVzxB^WPRo|MS8a-oZ2##xjZ5#Xt z0(nH-A_gcAn~OyMGnQ-xxgQO&5gu z3zh42k79@F#c(kuIr&ZJqYvD+mw@B_Rlu2M0W`d%c8MJ{VE<4S_hd`630*bnNMR>s z^WURMyjK`BoU@b=5EaX#*tS3__ZBIy{7LF2_OB<+$B`I7MtpyXOXt@ro)^dO5*Wvh zqFRGa7i5DZ5FL$Nt+#PJj7xOY&1I zc*l-DD3RSGG^hX4wby;BaFcx@o37qFY&pnzJHud~QX`z){4?Y$ezLSP7)UUpo1d>l z{;;hIrIMGm5&y)dn#5w{@ow4b-?+etr*O-pmD^G@xz$t0zYvEaw(M)&I%{{5{7C|C zu8s_j_i@&~oZq=I-wdMJ(Z|VM=?{=oTpw2n-*8x3YlqKS&P#_%u>_S2H(^&zN^`a3 zuesY5DS^A^_`>7s^FoNw;zWEMcLHn?OMLU|1*~`woP9znp}9Glvh!^@^T~Vjq3SS} z`g4?s?RuI`>`g+ydnmDefKeEy)c<@M)^$6eUMpyEH`ZktYim8NHnWawMu@$wDCq7y zvo`A!pXb6)*-JeK?|ij$CGJa64u8b3*RkL^-=grjxMYJ2`x~oMt7^6WR=(jOt&qO| zJ1eikTL5cyBWSC3Zea5gdeHLTN>u9xZPh#h_BWQD!4p!AJY&TVzWy9Xn==;m!wc=< zKi???n{OtL2G?P3(OUk&y8SEF2Zv#WG2wdTtcoeR2xyepJ*z(Nb2XlkaflpAceP5A zY9EE|%O|zdj1+Gog6t=ZgU0hONH*bA3P50)_b*z$bnkUJvd73)5#)AGg|6S*ITs!~ z5`%L4X^Fh00&&dX!Fy4H23oresXp6B(S10?E zKgNl&HSA=bd;T+9n`x!3m9dyn=o{I@!z3?2L+uGSqA;>=k_%YJtee>DxI9}K-uxze zbwk$sG>v8W4TsY`ky|E~@)*Z%t6k~#6MMOaSzNM72W1xIB3Z!uJFDbd9#BRWhZ#M8 zOzya4r%c3-Fk7o27lJGNGw5b@Rtwgy1?% zsZ|WMx-N!1~;!sz%Y6VXTvS?i8^=gC@S9X z3*pkQd5uJANS0$w{RV37px0*cuaXB5VBIa%oGD86xA-_$g)adkSh^_3?0Pa4VjXHr zeDBRl=vaRJ0xNSfR~`&3l7)!Y&Ip`txy0V7J(u=LyP%9-bU1cc6TUI&IIp#kiSJU~ zpweO=BmV^Ut!S@)Ca-l_#3L(y1vl!ftGn40cXI?m?s=$_{%99sC^q(r3)4m`2aLGH z`M`*s3;R$|EHEbco4gB46DqUdLEE|GlpXrpYq5f24!0-(4g%|q;1R7sou%MW?vuME zUvN~nHjq=??GwK&)MY=0F!jz0HzaQcR&tPuf<(pxY_8y*JF*LzLgeJ+1GUQ4rH3rW zQeCMmxNYjmujLPI@6RX}X)1N$5HXH0EwHNuN;&|TEeNtUWFtnOOKMVSFaJKI|^f|0ED747>Q#cmhkd31z+ZZ7n??rN#vqYaQ+>gz{W z^?|%Sm-1PR(E{p>R%M>=M&c0eanX?EZoarQF zqTF37G+Z4y#uDxd3=t76NpD6h#7BCV_Bg(OUEtCkxpn?*tYFv%ro#=dM8QUdP(Q6} ze`b`9@N40`IF(CBKejfK=nmC0o6FQgDs;&sEhI)39#%PG?k(SJ{|1Taf-s!Y7d~ZK zSQ+*S_*vsWa+#berK8tNR`L8zPwX9JM52Ui0X>2M98G8E6lp9M!tD z=4zku7w^yTNvqR4yo_m&>S73jIGl!Y8$QyT(x>FBejzN(<;45oI(sMc{Q~tte?4z! zdGfLZ9e(3F%%xl8@cFl3G0qArxPo}Lnx2n z#MCcNx^<4~nD5REuPf*7grr&tz_cggqsq)NL@=aI_+CPU&w|!ltTtiQvm|}YpKbXg zCS!V)cw$hz-9u>C@cY$PNMD(lKf zmJU{s4~s~kX^JtCJW2B6qdBMBi|_Ftlquc6+R#Wau_!zkLDeuL4;uS1F3{ObE%^Ab z_@;X5LN5L-9X_-p*(<3hsh5(qYhd{t~HvGYZAv_ohB3h3)v2(p#;3R{f-I zTb$cx_aXPa#{AMbjgjGX_TGnJ%DyP<61$G$#td``7>7+AXZDlbYA%tfPmWomu5I_3 zqTezl|8mUBa`6*~`M44l0-kc5_c;b!nm6JvLj0HwYx*+`%o_J8jxk9wE1g2PA}4o7GO)^PY811=f|=qcvn0I?{0@nhXEC%uQQycpRv#f zjhNkfS0U}Go{8a&1GZ(qR~(#pN?*X6Y#Y&N7xpaW%Q0kr$`^jUtg4Gbo4H_vk9Xd{ zr?v&T1(#nv5iFK6m|_bmPwT*_*Qv`cnC|tt+)juYzM8c#lwmp3BW^q|pT5!5@>NmY znvmwe>nx=iCs2kC_t|Yqbo^lJxzViI)d`VDe$`0&)+J`Ku9!oVoMq=+CVN<{dm02o zkJzR{gkFlIfg^yhUKeL{6q2*eoL0{}hqQ39nAV`7Zg_weP{!sJf5P>4b^rLwXC$-3 zq;+|dyRtW?nJRW-r%gw9*c>CYo9g9gQ;zZVy`^$Kp^*DLpEsdSNUB_guc%-}Vzz#cvLJIHb&_{lrYv z$Pm%TqnCC(heKl?TW{&k`Seou;9KYsv}%2_L&bpSKv1~7+P?O1j0?}Y@?7SutxpK% zGf^Q4vnOq}Xx%=LByX1sZl);}U(p^|BnkEcu zN|;7*XG9iB$b7cE5_Ep?Rxlc^q^MzU)haVqmo-bjwa&-t!s6{zp*W!?m1l$u=ys}kT#mmW2P@7QI!ea_7#blvY-rGR_OY|Ui zzYpJ6;uTY3Z@$O?MJhiUTEwGIoUa^7z>%B$j^-ivB+>T*hm9C`C1i-8w!$UWI`~_k zkh|~1&>HU~@lml34k(*bD7QcgkzX4U+-0siAHYpe%04yA&=<>KBoM5i_?5rLgwUsn|6?eBx|7otAP z)FJk3@-A#UAsuGQ$S7baX@bs0*FrF=4e)y<+>r0SA7&oCawpC^4n|7@uXR2-vw`ws z`pA@P1>&(`R!=V?X72PBf-Wl$C9?9t6JHS4oXIN#_E&Y%1;-zKk__4I!+tu2S_@Yb zn`O#F9&jc68+!cLx&wr})IOn#x{?-k4Sq~MmyBv;H&-clzd{>4&W?zsyC_MnH*2Q;z1H0Ac!?}w<2rVc_d zj!oIaqiG(i)gTMa76y+mNc+^&9R0>$vPuvrLEh1BRG(}B(WNxWN=OqKE{RcK_XPWG zox`6X&Y-vVGG zGw8&JA*k+$CP3py&xaO(>})Ao=C#+yFN~I&Px+!-)MMA1iYe`y=9M5-y*mminT2Tb z)(&5ttGwoJ-Z3Ex^R=I#~Qni-;*Xh&+n z=^S?YMEKe>w-W>4$W&3roMXeI^X7;gm>L7zc!deX9W7zrO$m`1zKGs+aS*MkS{)fZ zv^{kEQp_~UbZhmQJMpVIB9#FmT_c8Y$IHHblBQEGYUN{tZKF9`6Pf409*uM5Rxgr^ zV~4cN>|c7q?D|<>*p^u{+yDW)-dL8`hs<{VUTpro9Vh)(XW??xHE zuh4nO(HT~!Jr>1$n6SChsdW_e@N}uzHfL>uz0fWn^raprK8~s&LN;PW@1gv;Xg9M# z)_Z@}Y#OspNIu^ZAAzyG;A-X+U%XY>Kzk2;iMRG^9ZjvDNm=DWFU<1Fu?Px5RVCOX zEMQ6KUP-NkjKGd=XB_GqPaeF!T>&hXNR(aB{``X<(+g;dw3h$7i7}Y>>^TXI0FS{p zgzqRu7p&tly18SYCXKOiQRS}T) z>P=X-qX)V!sze&ah0=)`B`BLV9`h}xa^L#0)MLZDfI|(}S#NyiOw2I*b4CkUUY9C0 zvrG`kp+m0Ejl7ERIS86u>FZd@>j7)1$f6pCAD!Ay-)Yg=I(UnguEP7KS-~%T7FeIX zTiN6*SBGiep#069r16-hV~^kvSPThFz7sxNqujJ!zSHxu6z0n%a9zInw_@I3xg{*C zS+I8BL;by#v*CTssHW*+ZQ}3rbB-p;#5>xNaCH+&mY{oddaJe=m3FA^O{L?l$^sBY z#Qodsio{L@_l*nOa){n9SK#sFYOCHPVkXX)s^ixAYAYdQmf&~gg|3^>3~{ioZEF}6 zVxFlWbovDooa;3Fxz`-s4~z*}m`YKiU(C9$)ON>8l_aU-h|k5o?4`T+WX}4`!=Ueu zR?CWNzbij66Q4yM`S<5Brv6kfUI}KxCgl?!)8XTIPTiq~j80T`OH(sPEO0%=KGDai zQI6B(t<#MB*31�V{4$zf9ihI}Dhq`ytma#UA1R{rxuSa6|Y&jqgi30Jm5WBPT_8 zL5G>jkq!RZ2(Sc{%V{mMv(18^6*(N;3JcjDU|@BE$kx9<#lp-zjz9Cu2AdXwLE7EK ziC|>++1&_=Vpr-XX$>g8>Qm!#f?~ofqpwL`XU-C~SR>NH|H}@{SxYrtZL%ljXCjT8 zl1In-_S~Uc`IPyF)9}?9G{V+==2x+*PQM}q4)w9_J`({EZlSHAPwKm;OL8ooJYEyQ zjIuVcVNHI8O7JdGRo!>+zjgt@Z_tVAJEBa49Gj#y=X0T9S9|UxI5okY=0@^BTfywn zE3ka{-du?kJ0*H`%TC^mir=Nm%2{H2?mS)=vNo$k?3rJm?iWqlVdsm9FC495RCO>_MCclP^Vw3>MtYh#O~a8!Ih|z!i>A4YVgEB1RpDHN>c;ZQ>jHcijAl)8Y?>l`tm!K}@ zFp3W4H;8%Sq#|m*s)2HWzDsmB&)z_C<=G6!aX6SJHZ)7mNmaYK*CzS+qFpx|yQjNj zxa;D2%%D>}L}ZRlJ|f#xtOuiym`B+<+Kq?e>DEvRhi#oA+UZ0Bhjv#|XEO?!TPJ$q z=>u7Tdj>hm%CW0T@H+RyrKYT(k|9lJ#Fi#DC5#rCj1te{0} zqg=4L*JUy$^J?pLvBv#sT(7Wm1lk3fXX5lw$BVDgM2V16!bVTo%=9tN>mp5_gTef5 zwY_lqNY1WTI2EJD7n!r*z3%4v;V-kKS3Tv$GgY_|QgQCJTm~nk16zjkBhOIYVh2L} zQhfLC)NA#0OV_#>XD-!{+Xvh3b*s?SeR@1urnI|iq!P%lEfAt3$#Nu9hxxK1OJAqF zGyLE%vu@>pR&)DfvXFR@avhC^%dIio%v_JnrcB$>kyhM|<_6qu<9#0PX9b z!&T|jzyupdd#HXiEh{S-v)$*WKDx}smpJT&DCvV5KROSnTjy-iVEtdzs% z&~mSb$VT@}|B`3{%{Bj9JuF?XV43UT-Lc=7(tz7=Eb+4ZIPuCQt`C;zVU(rCZh~2I zwkSw#WO)?o70|(hWwwrsIqI?QfSPR_jS4WVRob%;e6&Rk9vQ$2tTECF`c|hs@JkTe zHp%lqr;JpQ3bT8~u)toQ$PLXjP)jP1jyPnYE;M`#$PK-;NL%+)ExoL_`C!)1g)_BG zQZqp$yUIPG-ZEj?M?mjJdhR(^)Hi1m%(ZaGnR`BSDkbhY_hBv9v7VoiPd*ONvd<{r zQnCM#*`JmQxrd+2Ie(nJ>a)05+fqmAu^9`0Li75olEGqsjSiPlOx@mS`f_#*S`to@ zjjqbhR-SVd-kGVX-PUb$7a2fL=vx30b5IFJ`=wIZ>Q%OlEL)J7Eyn1U!vwF4yfgV>l~?kftk=`Ad-V9qBNH@A{v z0%;OC+Daace#%f@A3Vas#TaQ(uIv%6ny05e_bn`?x%1(e`?^>e-*C)pH2wO~E4?cF z+NR)?gh@#Xhe7*nDvOs`*wjR1_kFh#|1KE6b%Do-Vs5CV(G^Om)Ju^SNXC0HzmnY8 zhe{-EDy!z17~o6_M_XS$M;sn*9=OX(^Dia(%N~94_kq6jkYE&fQiyX=G}NXFISegS zGC7?RUGG+o0V-ib-Tc^1r!%*F&2`Uqo~1idRYV;~n?MwStn%Us*i6av%Um`}_2@oB z6K}3-fMuMc!_Js11=egVXgS*jTBl=Z>1xbS-@O33o8GKA*{Ek$;OUIo^E_UzHOW#;eX43+D!k zahI1eq(hc{+*0eumoEy=dOWsSuf?=1>(C^ha;;QbpGqJ8tx}EPjn5<)umlA$&*IkD z=UrP%tNf)`lSFsi=`uBv-`3oi)J?TlS^#Zzp=;oZ$!n;OG+^^Pf0gJiL2eBFHc6cR z!cc%B>&t2oXoP4>;T4WSf1VOal*4jxfIUu3ITePIKGAa9115%@7X>_~xxDiAKwz-W za$QMSvX+MM&Pip{;7 zhg$PKcF9u2AG|3~4ynwe?CoxSxIv5zU$35Fv6b}Lk(ER(Ed~^JYb$O}qW4&p50=!n zqQvZOe*%)b0QlnCECS&A_m(z{2;UgpOPUBRm4A%|sL^fDt-pF^|05>81;O1>hbe9@ zD>Y5n^C8;@B79ByX2nWWQ$%Cp_#6Y9R(m;+eCa+&+@wbZqp{YM%>D-lJI-iW5UFWp z-c86Z`syHNIJFfh{#|V&(4xtoCW#lvhLh*?S2J&77D(php%v5@qd8|bakzRXQ!fe( z9B}p*w=R@~K6)33Hh&Zo_Ytj=bo9|IJdQKNDMyWz@g4ITSPI8aX=nF{DetbMWXI4h zrn8avFV1k3%=;Ok#TTe@iM`c#A+R_zp^`u1U)L6G7dPk;F87OzT|Er} zfU9RRftLG}hbuBWC0>hoP8y6aEn<>!K*3$Sw7rFe@l3a_x#jo6&1!#};63GxD8D~f z^x)0Gl1CPu7&ytTpDh;NK}m_}H~VU8=DuNnOD0ak)V`@KD|EO?c1Wt-*8m!~UQz)c z4M=dL%Aa<(JLFyI;#?A-$=ByYo zLmwW662Qv4vUMBYAO|{%UeDJW{avRkGY*#MWDk*sMk+&U!o5wpi?Lg%NzzC8oabT_ zMEPTCL)!{^=q~Q&GQT=nNUt?|C5O~c6XD09vJjozROTn!@a3P7tn*P#!%%@ zq(aQ>Xu1!$pF^3+Wb*O-@l43JX>)uvJZ~W@*q3iIYt9k}A9&UE6EnH|IS-@;W(ejG zx9-}ljp82+d!~|{AZ*>drXX{0!K=poAU~4EDH6AdEk7;feJ2y~>Cc@!8Wv-u1uz01 z%YWmWenU{gaKEV@m*IT^iYI1$9YZTZKPvf5rj+#7xz6-vKqWLMN*#Q|$$D<1#0PWl z(lBl+Oxd$S*+?0zO;+~#VJ4zlM@Eh*yHOttHmi9JHO&-=1B*;EOES4-*oJ>>s}7E0 zMA@!&1H2QOrIb(A)Y2qdX?@Y5G;Jb33n1x(4C)x;K5GD7yiYwf$65bYCzP|Pi}qTiaZX5H;r zi4KPwEj1P>pNX9X$UkZYz)v9%$1{2JwZ|9?yba2Gq1sWZ&ApmQ+i{bkQ#r?XOS5{C zTbqv-&5Vwlp0>E(qW9%{{Eaut`tp6xyE{N_^U+|Z`M=j_|LJnz95*{GTvx6f=eq9g zyBtH)U$&{#E?lZ!=;}d1N?rTin?vMDEs4Q2HLm;hOt3S@I2k(}9Ua67Y?Fo5=-6rF z46Q(9Ni`R0nYi}hT-$?evD7)*eNMYm=V}-T=)FQQ7xmi{Na2Dq-&z9)Z&MT8Kj6=B zc*m(`K9=3aaB|u@aZ2@)iI$A$!?-F(WQ=u_iXY>~g!znG0@s*T6RW!PZO0bjrrI@B zWug)8F97+V2dDUn8RtZ&f!^!7o%)i6WkqAo!tbSRzU-6!wH3{~Gv8iiQ(3*Fz#I1^kiSnlf&F*t z0|Q5rckn|U3hEV#hH8hKk!F};v*tRja%QSmzE>%ir%e)=I48^TB~vodO{N%_3ZZcr z>N5C+)d@{9MeIVZE%Eqc(5mez{-9O@8t!tA24-l{~`Ur2pLZu^b#Hyh)34FYBFBIQl=je@2zg$=VP zF@B9Y+EVwJfP}8jJ8+ErFzEYhpdID0bq}upk=EyU-21c{G~c0y+MA>@imO%D<#**z zKm>bdfBcaAa6eVD+oVj>tbgr{Ez04i=x}J+Jp!zeE0_P#daTi`GC$Pa>+;d7H3Y*E zVb>#Uq!qM4DvdYo+xS5yO`sq$6I39t23%r%!vAjzY!xr{+Y?y40B$(D$IA3F04)S$O z`d$0@v{p;5NcF2J^($#lY(M*v!6vmL37VsYWNMNZKw|A(#bx8RcSG6LxEP|@neSsBI^9Kp1X&>sOnj4ZDx41B@{Q$ zb|K}4nxTun0wb|gJy9ufRExNY_oW2qDAAqK+Kuyr<2rfzPddds&}`fFi-KD2@kRqU zcyBzI9Uk`zM{nkE)VU}MgWxba;ZtV1^z)4P3Q8aV^@ZJS9DCmTuKNx$AnW~_JElqg zQ_K_7ia|mA=~d5L19i9q4H6o~$#&+a&8N%>jXt1TFNCk4_&X6W#%;UP7x~%GIF-ie z4Ig^AjhF zT-H*Bq*&>zB$WyTf#)Hs!C1pp#WG0JU4ZJt;$f!u7a)>BAKf~gc8!V>0SVfk-R&i! z&6f_RX0@d9=q2LkT3tTk(=;$?HBCExyj0OEM$MeOqH!q4!b!cIWM20p&_F)9R#pEx zKh<^E`tY}A!U=zkW&XWD$m+N#W4nsh?R1AQ-Q(5DnPK^e*G$hCkLC^P3rdZW;-;*{ zA1=T`-@Ce^0mjypc=GxBPv^QsYnA5|*JqTs?wXPMUSvd1w&ZCcUYc)y2hw)N!9fe5 z=x_3FjJI3+LBFw_H}U?ELRyT}4>PkrNhe61yYaAF{E>xtQgP1%0wgo;pCp}wuntPVBPErKK-N%fBmrIGcM zY}&l>@pWM*(>$f#?+mWpYh%U`((**a)<9}asmGcD@7e-;^Lg}-@-NJ5AJ-?uuRLpiXzwFdHeNNp>|WUpRyEm9O;boRPNiq>7r1Zm@WG$Ptz~m zr~1Pt&=K(hYUvSPotp$d4&0V4X^=kIp4cF_bW$XolgTQXV$c&&Wxnsty{v3y3pVp5 z`~(oKTPb$PaQy?Btc3j8v+>IAz0={cYH9LC)>lPezdbRQvu*E=T|3iNmz7GsTMysF zA+e#Rz6&?UbK-a^1I~&yE@`?hUPG@uP%XZ5Slzsny&Eb9jJLp`rl8Ev{To=n&oumt zNZ>mJ9n|Ee)VGJHx*^WR-&!W1*tJrZAMu-|_BK%8-hdtV3;VzJz^msGk7FOr520X4 zSuDz;$je5O5#o8iOXuyg>a~<%`K`7h<4!4d*KVt>Nyh;3(WWryN4l(Vxvo!(Tx53! z?h9;toX0NRu^rLGx$|SSemfak@1)XMmIV;jC^RN5xsW?X!Rjv1(H2WWM=*U_n%qXO zR$W@0lPY-CR*$aV+YCc7Of=1rO@yE!&WV=PIS8zQ^8;ax%J&r?H_$gio5E#T?e|VU z4UWunnXmK15=~lP5*!2QCRtk3Fk~x>V|K$r}p7gHP~Za#qayiueQ?P$=%bT=voZE|W$E?@op$VHlJD zC(z23)w%giOUQ1(f0cOa1jLgs^HFGew`558~<{M;UGt`{=*+7uDe zPl4AjD94qpzT40vZj<4~^lnj+SkfN>I&t##NnZ?gtA1%4n%4Drck?dqMIHwdfD_rb&${_X9a9>b>x@-*&B|XJs~=;eJrQ_IfHRcM z1LVTZNFkVvXZnH3SW33%!RGjV@SSCl7)iv+jIw#~@E2Lk*(>rB&pS>}MRJU4*Wa9> zpZWl3r$Bq~mv`K=me7(Ih0n&>N6S2ih}Odkjq?1aCT~P;0u>IaPzlvL*B7%l_ovM# z>WxT)0e4j!9#is^H)1LI%aenh)y@iWKejvdVc&ncq*37IL5l71g_M&}azT0QeZN#V zK7V`Yjb)z<9^{PR_PJBeT8H*n9H&YC*ym8LNAXqyE-gX+ipi1M754|p^~}t5SO#7$ z{`JKmcz>l|q0#%W)N}V&y!fT_Lfg-yDMxh_N-06M##%L+)7RqHlh1|G>`Ms;>$DpE z1asA-;!b4a{3SW82-x(dSuTc#hBbLETM&mk3jE`}`~>KAx>N`$W=J>&yIP0I$X=+s z2MvtZR0Svdfv4l-jfu2!Io2)P0qsp;GgCy76{{83mSeil}5$U3*;|uBjx*&i&5WA z7~dloTvQ#V@$9Ea>}LqRdA&TMPu$zSWS6Ug`1AurqGIPv z0?vWExtd@25$6QMc{ zX50UCf(4|q*Tm0b{rEk`dcq+wUB}te*|-*^j0XRPGwL4KoLP(ZB^eL3Sw{)i+NL0( zeS-!LgJL9a@o~5}mNMUA6EFTqF5fJ%6_Z3m_#*`UJJ;Op8)Paz+kV*H00q_%9^L(v zlMHA`yoE!SAn-a@nskE#TZna@fuieJH=CnY5Ss`*yUp7+PW__h0bB*&6HzTRN+y>5 z&-%v=3V;T}+O4&_&e;YzN_Uv%t4PLVP_A-2u=Bra$=^7bT|FML<@v-{{(I95a~KvS|5w@g zM_$F=6GLN5JQ@EuT`~qk_jQz8Y_uwwjMn%?AZt(Z{^!20oFyKqwhqwmS#ZAds z|9Cc>?0PxPqf-07<&FQiFgPKES;8P7Lb38-iRaI62YlNWjN5@$760v&|5t=H#sGH` z7Lp3xmeR-wbeB;6Q<%4})d|me;@;oJ&p$$`GoME3+aDln5Sp}%!$R~=NgaR<9wG?0 z;C}~6|5UQ?dCk>x;Hy^{9$Vlr2JG@~dyLaP5S0nmVi!#gZg1-!R32QP3^)l-GV4*~ z|9l*L^;WJizJ}bN#d|mh`Md@A^QvX(zt8IZySDr#3IJ08qNaXtAM`)}&;KC^vd7n) z_2=l8;=e~+znfFw^*SDNj&kDvtq1=+A%x=k_TQ`F)c@-ui~l>pZ{+45&%_G`B6(m& zhv+|_@jq1^27Y?2njDfe8^ETP|9sr<$N%F4%+%Lr5PWCqzx?elQT}$qN0Hay;{U&h z|4&|oC9yndw?$%ForvxU)WD_xjG`&>!tB zdmPWz<`34i^c!A)!m_i0{V}(1->BVNluw=ZTCC#@0k)4Mj#&UZvB;*td3hng`(NZV z((vJT4`~K-UkWdFB}hf|>i@M|YoA^Q)v9~IG}e} zG%+InZAcn@Sdt?u1eoG^i7Jl(w@`Yn51#E#v@?l2RH@XyUv*t!cH8#ic3J6K@RZX1 zf`R?nC|wK1UFJWtPUOD#Gbgo`0duQ&plbT<-}?cNO)w=t$YT>Hj1k+#5|^N>_uU~1 z^H$f*x8SGjfI;O-vHnC2G+p>OZ^LGRH#Nj&rd;opVoCL)SZHth=$L&K_U*B_GK@uB zm*9P~UYaUVZc0&-mcYh4++DPC|3YU5f4cHz7cd!hfKM;!#@q@FIKBY(Z{766 z{lWO3yb!nzyh)b{X;R!&b2k({p+6JuRY8rJfyN8NT)-s`SF z2zV<4cg?=)^zf&5UsrVFyZK-IOeEC_%5naFLZjmx+f|eCAfqRW_sVzq`pVyaZANNk z;aUTvtVI-mxS;j*^dR6AC;OgIpH+A%qXWJgn$6KPqDdeNp-C*tl-sv;=eOlno_EMV zpLb^yr_NpoY%05L_yTJ*WbPQ_RX68<@_X{+$*~aloS1JEYV245Yx1DKs6faPBq4}K_-*p~k{hk5%Juq|p8 zdedn+{?x%b0JE?TH`idMA-VQC2H#WpsR-BqvIiHqvTC42|(KixZ3)Sr{| zBzX$c%D&gQfs6sHzzTc0Estut`w#4MhaC6paZoTzmWIFsB{Jf38u(_T!y0;;!5S9#b&-CzHz@3PCOo%y|8pBEE09WR(C*b$q zJSg!(bgO_?&KckQ)Oi#7-~JtQ9Tw2Pvf|nG^}Cb@Mtd0e-5&c)gBP4u=neXZvlG?x z1|~B%6V<`Nq)pV}MTZFmhnbpL$RGIY8*iy^t$+C8`dW8p-lw_7K{wF{vs5^kuam|7 zJtEkv&!c|Eycd!jRt^ds#NLa6 zo#*O}GCm$Z3;5ZdZLY_>1YwWbjyy<0NoM%Id7mn&O9;5c-Q)9G{p1F+dg$i&yl8ff zM%`E1C6TXfg@W$Ci^}>6BCuI{io3tk({_pJ6DeXRdUv!q++~Sdf3xpiKyTtLeZ-a- zu)GClG#~lsa9uI`ut8XO%5l4@=6Y4Yzr|+=4wh~)xUL$fFGXe^n&?~*9Q^$bInEeC~CP@27S^%2zn0R$_!g{Ta;pL&t z`2hpYLG9B(j1P!hm5X0m=LaaWRCE)eE!ow=uEp+#1?iJPiMi+f-V=K34-yfu{yQtB5Q6Ryv95k^1VJV!tP`$-LZ65SgS7tVxzpEzmvy?f>BG_c8nyv^h=i435lLvQuT4B+xV;xg8 zEiu6h)1Yyn&mAB(`NkENv%E4xI{fWm%tD%td@XoTw9&bgliO4TAct}`S=+b;D;bqo zGp(}Y;>oP$;C6{!iq?LQnv>^?5&e=TQK z;FaLiJ8lzjzU&;txKaN>5bx*iM^c2fMur$!%=L?eS#=flpK(ovtIrq?&^6M;uMO4? z=Ib7u?WrgTik|J+8#S34H;U09S7-XQHOx^?--5olL&gA{9@TGF>M6jM0k_DQmSAJ@ z6Tgbazd_GR!C!9?-{*;Hz*PNM4H@e>FqZY(lR_U!i}_l~-dvIXnFO(DUQdzneVcam z*;6J6GiumeTcmz(hgk!)=b=!!zu+TK(Pk(=;p(&ahcf#&Jd2$Twd=+e&d3pB^8JM4 zSmZ{XOaQgFs%A?VqUxHu5hnqgqe$t*s%f5)nzkhD zJ)eUflN=jdG=-;{TBIQ~;l%fk$9@SAuxz|Lpd;YSUptS8bexL-6=LrPBdN|WFH%7d;4U26jla4S!mpm z-N6Y&j#4gh;1C|(2m@O_h-Q>_RhY={mG+vE-*03f?Y?2Dl^$!a)E+I>0;k=(s5)4x zAr0{xW%5UP;P{5}DPUH)f!2>j zCUtqlL8!)4?~`VAG^;A2jav)>pa$qEpl_R5}d8E!27S0|+{VS7S_Uv^7bezDo zU7GQh@g!=GD>>X7+9od@-zAL~nlWK@uvl=klho5u{@ao}j2DW3rXJB6Hay!Ny`}=! zC+$!`t};XvsskzFAh+tJUc7C}=w4&n zOyK`iMz$rK>1V#W*gaC)&>}W{y+7*pQx@ zT`-y|&1z+*Ky=h{HR%bbHP~C+ae99Wj(ci~XhSyB>I*(+dxycjshxV0_1F6YAdBbw@@(~PjfnDosTq}|Ln zxzUWS=hQgZ9O@i={^!4fDBD-s##EbS2`etES90N-#igT}TR{&YrX8a0kVMdo{7G;n zk&p|07Mwa7tvDPY{s}q6h z{0yknz={JXQ{F^rb*y5OkvF9c8|!O2ry3ldXjnKbFXCVD)L!P)uRIojwR{gj4y8`M zxS%&!?O5Ct^j!^>leQ3VOQT9g*7<9Jt4ODlFx&(i6Uv&^0cU6t*PML}fqR|j0)~LE zjlTKd?lMixya2Z3$3Yp1MZf5qb5LWAz!o=4lR7YPXZ+bk$n4s&_%Xt@P+#)UENum?%WVxC+{79n*|1R&h40RY>BJ9 zB>aWg;0tlBxJpxzJb%)QyQeR}jjdz}vQNg7;LG(~53VQa22!ivaexA>7aMtv&cS9{ z24Q^~MY1Q#M7flE!Kj1ya%@p1xg4pBaorWdy5f+8-!6;d$ImAreLn`cLWJfzJP5B% z+h|Ls`T1oWFFrrtnJ=S~Nk3uDFlWJ(+5Fr2uusOJJ&~bc3qlzt!mOlcW^KvRvg0ir zPkcXJ9k+e;gLeDi0RwF<>{2AP7X;OwKgO-wg54A7bILTR#$gP?Klcgsnn`6O{5H&b zD`I$+*J&tzE$I9AyhY*=IbFxuCVB!y>mnwz+KtD)iY2`pf7mm8WN%N9(>;6Q?5uK@Ws!cibCf$h(&NGJv*j?NmqCgjHEo-0b9X z*J+1nLS+42N5ka{Nal~AKqt)J*g3T&ZWUm!_KYDA0WAnXr(Lax>EEoV{23-ZXX10H%Ectf-I$qu=;zvg@*%Lwf~ziL z0HLYNiM+=Pvn_#r;b-S8aWuc+`y}^pRqjSukr@OAyzQ9^w@G<4Y~||V2!TS-csPOk zT3=wFoUb8Q9=s;%Ubz~359q|`35k#ae0Ntn_Qo)IJ+BS&9tkwm;Af--t)MKZ552H5 zcnHQtF1und50D~@i>nSc=pu(JB&lH?i=6&V@1v6wOCqX4ixKf-h!AxpEjv?Q5DkzHwQE*h^sx zoE{Ubm5mY*e^9c5&h`l48NJUto}TWR0;XCP<5uQV zhI==wHsmI;*JR$F@fgsz-M==s9jG%AFDe#+zsMA&!y{RG&d_$+70XoIVt5Rl=AWnv zG6c0Ee}M1euVGbAVz^BkHBspjTY6Gr&DAB_ z>tQG?VfD}#5PS8j``;@#YKMjqh9XYd&KNgf7#v}bH@Q-}%mk%ogU$2$G|9J2KCh-v zvl3TKUSi&b;=OkB4Sz`*6G%hu59Km6#+In^Vr%QXO?*-O+w3E;iSm4K``1H3)z4n7 z%FI{7lAeXNxh6e(;#l5-Px;|-S*WDB?i7cl>M&r;UYdrv2^*Db>_E17{*hx~p5)wb z?wr1T?!?jJB+U_)n6%>Vx_)rx`PdC)E?{1Nn~gvKdpC%QA5&TAzVH}gqipLms8HN8 zr%|AW{38gb?(my2{mQ3z(Z>aa`fmh2PY0asmaxn1vhG**)Z{w&@^j;Gz^H_c%5fKP zRh@38-(tdRS~*yFLC_=hsQbaHCQ%_3*k>my@$Ls^$THvam@@yqS#JF9%r*y|N2dU5A! zV&0p}?M9ul5UCJS?9#V`Hr4quhYuh5I8q10FFqU`9E^m>Sd-Qi ze`6&q-9C#jLcE1Oojxt$qu~tG7_``3WD4OetE}wmgy-NAwo9(vnSx%?crRqp|1G#! z->xq#HDV!^zap8H1&jbs9IPsD8~1g`OB>S7y$K`x@D#K_#y$d0$H{uHSMuJ$V)5EG zBNa`Tjd5Caw*h>wA@PoTzx@=a__!Fe#UgkGVQm`b9JmZQ>wF;ltPs}w>~2_vK%MaE z4*fD}U^kF}OV_CU*O@rEk&57p%e%ssV!Th-ug>!n5U8JfVjtc3=?Izvdew_$JZd{q z^@j8oK6ZKD6e|R3AP?DVl)p;Mg`5i4IV7yYs&5Q}NaichomCmqW__SmPEDYqZC z7zHD@iv?3^YDny6Q|YSiUA)qDNRX~Y1wUP-xFqu2T^a0Pedi~nb!Kz$BDos8Z+QeVBmoPd~SW&1n60ZT@% zLisRmL*5vp;JEkbeq#^Of}8*|&-5*^RgJL=rfe?D5bOEvE5;7qw-^eNpV*{#h)P$9&@Z+Ij|i-KB1qP-)UNQ-m4<6I^>8_^_N)8A8d z_35m?UAO|wbCRTYdOvGde;N30-2x+88+~)&0Q(GYbHwj3qtk;==Jf^OFQQaHeqlFP zDp^pBAW&#m#*!qP00WMCr?4}@b`C>A@7ufU7Kqt7sKp>a`9>13<#4p%{$aZyk^x6* zYdW5pN%NFv{yIse*n>gFQUlL@l(uo^lkbX0&E;W`u54IO_Lp7@9?2~&2B_@imxJc1 z&&%T9rsOx{C6^>|)7mqA+$7{XHA&X1vjfd#iJ6{--qXC6r74YjT#(CjIE+2FS{#+0 zu^ij?i1||#ZUgp$hAu3u8wN>laj&$`3kD4*nsGjO;&tI@@0IxYWqCwUl*AeB046)H z;}jqdc9Gu9ZaP(t!u;sJx0=|c%oKP&CgxgOBKT9aBm@TeL7kY&`MI&ZZx!K)OK`aoTdt4@L z&XNGgC$D|Q5x9SU9M$}Z^6Vu%Om+`6#R*XA-V!d{IYuE!bA~aD<4W|Y6C0h<4LY(% zEp44bk>PuBin@C4E>^S!1?E7e;JX*Aqv8orU~$~F%qyi{kqITl3(V-(X8*W>o`pGC zb7iK5aLeHDYJ-cH>}sAN9yN0xW&wb^xCU*)8R>oG9xw0|c&Y%fr=tZ#BSe91<3IDN zP>6l$dWrKKy4k(yEsFC>*QpSs!%@NOYGKmh;?T#1Rub+G99eD(q?strrNH+9CJ+b* zbCR~ZEz7tM>{geY6Z+M=CwAPK+TmZ*Z0Isgf#!zzCZwT| zGki$1{ItV)?pi=YkTl(H!!G47KTz+lHsn0y^eGca?AGfdD)oDg6}fQ0scmQ7eg0Wv zp5i@dMJ`O!x-eMQXF><93FX0DbJuzV9hqGlvDM0htU1>!>g4AiTe=j4gj0|cnSWL2 zzsHk}GmO3LiJNqWaOn8Qdqyc7JwMH}5@-AG5^X#U0~jt1A2XtEoCISuO*ifzmKcW7Nx* zAc9vop$~{D$2BE*Py3UMfqP7;7=pHbH{U5gE%i_aN$-IUH$1DKJ9uviFuLG*cR-|0KKdZ=JpC&x z5+k!4YOK$$vUcr8SD?mIQif&pQqoP%A(lJs_Hetg-4%VXl3-@vHzTvf6#>5K2&R(p z+M=MnS9N)V(SOwf0F2Z~HnoXtFM+aPaQ@-FuX+v9*Y7I9{ZLT}A0@GTFe;^zg^GJ~ z1k|>j1d{iaX@SB+Ds_#qY%`wS+)7qWQe^a#dmZVG|1?kTVWOVamQ0dc* zjHa0ht_eQO1OeT*ydp@mLpN43Pg`gDOxnmRSl^}37XU4g#ZMdb9nqaA)y+IfA*PP% zgC9)g2fDSh9MQWh@QV#{lcq+kSfn&H6t?0x_1t0_X3+mNgku@;)q)A_?$Y2_CqpbJ zn@PQM_KJE$SV(Wtqjk?d)ZA#tEnWa1fvfMz^I6AKl9*?P#y@4BN6MeyQ+D-`=pwiy znEUQ;v9;)VBrdY@D#u4&jyaUT319tQgEiU&EqN7{ru0K4f|iCRoeNJXehSzuXTkV<^J<$l zc?9=4|0&oIs-hgW;H7HC7b(UOb!cfVoNufy8 z;=%lnZJF9_+TeqAj5zEb6odntfORUQs+V_aJQ2)_^;g(Dj~5p`S}M19wkJVLb)Xo7 za?TeQCN^*Y@A$rWEE+NN(~GYlYnu@3OWZ{X6J^V*4c?7A-Z=w-XfuDLt!w-t>`VPK zuO&&AnNt~fghhv9aQ4HKFI7z>lJ)R7Jrzp5CQ#M3NN#np%N-2p%9$xY2=pA)hP-}= z@Dk+e<&JI_;d2(ru`)IIRw&rE(-E9?U0Gu(8v>Q@a%~eAA%V2=mofZQiW|Gm9Ddzs z9d=#7#K-<(a{!x=L8+b+t(`DOgk{vF`$GlxrblbysfgGonnvq21@}bx;6EWO z)mgp_o4NFI=IxJrT?qGE=A92Z&ADDjS)L!6qu-^Z+PdacAnRc(bRUR1U1}AL!Q20`q~~_}=(Ot36V<)<+xH&hPm}9ccqQ zFbgVRkkGW4G#)A|CQB06cX(UG)@y5Mg_jj6V_&RbvG9ZfD_r7as=9M)zyg z(0a8gHeU3w#0@|GfMKf@g$~2LJ>&qjr7?D-@Lk__pFwoTg(^$#t~%*815SHD132DU zGf(jo8U6Jlgj%l+RHfJgX)jK^K13VArhqBF9(PWvkSA)@^FH`Thl$0=lNkxRa6O~O zX|9)&8_j`0U!(|wq0Gf3_Qw>aliAqaPiG_Zys7dli7xIN@02pGkphltwTAtT@>?~E z_tIqTQ;Jl)=!9y2vQm{9bQH6f`}6}J5|SM>RmA8Q>TGO(wkXNrZ=;zCNhS@cOqDg@ zO=TycKA*J=oTs`h?bNrFnv>@lW;i&qK*Fo1MqlNgInGX6UsgHXk7$RpX?OU0 zfhzdy?5HgsxRL(##|9Suj=kVR3Z30GMx^-CYS934ARDOn8p#L&{Lm0C5XhE?KcOSe zrQ1!;)ZAZEQ{HM3T1cbk+hx^liC3_=>TKW`1zl5z}D7u-APCWKCPaM|gv zC4<^)vS5x24lj+BpC}2?OBIrGk&)8(m)bhLa8S;2Mni{dlhOr%nxEkJvBFTL&`&6M zcS*{ihffT;zuX_|o^zgv9tajhhva=;sSE?Aa?u5L7)!c;2+B|uH-sfwJN|P)Mw(4O$^uax!ijuwVZs9I z;MXmV#pKKndxZsBye_fhPj{T{?pIrGO1M%ScgP!7LQq9IYdsp0$5L0NR6Pa@zyjUCc4>y&Z zE;KoroUZMa9)0}#cQ3!@i`>Y(Y1{UxMeC=~A+M=R)ARHFWhHMBGMi65t`b&OEousV zK??Hx8{4wbrpl@<;-X`T&u4uF3KVMc5xb9UQ)FVA>tt@OukoRv{>VD5 zVZe~CVE&}_x6@gG!tlJ%Pz4En*H)7=L-r37l;b8*O?xBeLV1%(khD(2!Lvf+ebq_{ zaxP^Cv719#xu$|fS4Vk2b|a2N%$3~d<2#epCFf>1s zv?%|m^5R1#;O>@3mpUD$zx|-~CLh&I-@2*mEi<}cDTDsw6rSIffF5fJyt6sgEiTiP zkYB+&@PbqEeKPRgO2^o|80nlE<&?BZB%mC+Kk?n`s2;ohuhggqVJmlJJ#&Nymxe`` z)t2H`NuUHwV8+_9A@IK=y-`DZJz}gPEtf=cJb?lB;y2~$!-Sc>3;dg?I z`FedUxcSi8^)wO&)Y~4p7Tyedvrzx65wLuB5@DaaI9j{>)2DBc5EhH9EZ1ww6LOkd zqHZ#c@@jz&b^GII;TL1B!fuEBdI-AxKy%S=YQ$#|Dr&}>DQbtA)2{V2ZgC!9UD5DW z{>v)-VH%bQ)LEU9yvG*P`zqlWEjm0HIB{hjL-=-O@S#_?I!Y0a7eEjcPP^g<_8aD- zo=+enUGms^3}};i&@3X!=(OR*wl5kik2ntFH6J(ssUGQSQJgm-buV~^iADCB=j42Y zg`vJ1Ex;gD>#F>{qi!~RViS#8EP;shLdB4>!8>ibbc3pC z_niuI?}yW<0T+zb{=F@awX>S;7pWapFsefaiQ1+{%jlW6?d(xl=*g>$)eo!BSa=)4 zKIs{C^l=Gu1ICwGTm*?Kyd1iOTf|1$lAaXDWbv9T270q&HG$=&N-$D0V>8!dY-{+Y zlQIRI{oX6LJ^fd;oF;uT1>t;+oB4o6pX9nW z&s5di;GIEF@EF5n0UR?}=_AVwv2;t+pGB2h$+d{OU~8pcUDgGjUQ3?zh$d}*zBU}h zOCa;%)sM?)DAYi0*v?k#NiUAwNxS*Ya5cEf&sQ|cwxy$glOfa%zccgXORS&c2Pw?y z#^gmiqq|)R-(wzcQ)I0qY-Aa3FX}SMC5@?(RUal-jtTHM z-!+Bl9w8a6uNs@Io*hZGJ1aQ0$9tkXF+%~TYp_(GJDoC{4GU?UHe7_kG>n}XjP}f-0X`X9kAP_+uDj*&whwg<#a;7QE|#6`3eu0`L?O(u%Z~Xr zpHF9GZC9 zeJ&avtjb<6oqRN{Ud^k(Nbi(1w=E$ zPV>5bv5`vI^5!A>NQ6(pxg$EWY0m`Y*xn*_bd+GoMTkg$*D5)zy(PNhUF|7#Oqr=~ z!1^z&=nY&l+>U^c?N3eUc6}t?Y-&tu;v}76xb+@In}s9{$T95hiToOEEBCY96!=kT z>x`+)PHozOkuK6J`KDCdy@UsjCXVRuYUVi#mP}XFtIWEN=1OQB+aYbXy-B9rL|#bx zd^n*$U!u1eQ;+PZf7Y(GA5yk0+HQ(0x(DJ8>2Qr$|Z zv-{%1ZlQcMfPoErA#<{bZi9o1@z#T<2}N$iE}${*1yLiD{6I1u^GCl<(;{qWKNft@NRc>YMJq(ct?93!^f}iy7nt*T9?us{&T=dpE zZe$6oBnONEUngE2X6LP%Yu05>PZ^s$HaL=PMV+kjg;>7y-nsn*yoK8O5yL9D5f#9D z*FPq!$LU^cGbNuct~m(y(47A}EZb(4;_EvzpJA(u^YYw6-?!N;G4JGJ@gMI+T`t`e zj6UyAkJ(I~0*wLgTSE=Gr4>5UC6xK-+yL?D5fxj(&mkw&OvCT}>fAl?pGanTUZ#s9 z>P+8W8bfYsIDwB}mwCq`CU3K1k%Tp6j0VLKQI`Kg`X$QvNY<2$e$IZ8;_FrobaPn? zm`LZs5X<(joEq~dNgR0QPv?z>V^;Q4^yr-}RwnY@0$F*cSu)(bhzl|hyQrU7oCTRw zpL$83-EwnK8s4MMX)?0LXLR1QLfe$;heRnQt*WYS2vrzx!v-Qt<^bPBy% zPzSgYa{lPKM}JvCxHa_hQ)rmWpiXmnRma2rR&4_;d776T4j{)~NVB@Gm@X4ibOoy^ zkxuA8(jqa_oB>78UJk@V`eRd(^{ibrfOwLn4`P2?ck-Nf>8+^eP;zaakqH-);K^M_ z7!lgYg;gNC_$8UtkWgx0lg%Rjp*2%(nQS9$*(KM)T-jIY)lWyUGt{~y>f}Rqem0GO zNwird!ypi>6Okarrn_<}sWWFR0bNU%Sj724nNDu3y58%va9 z(1x!e{wNlqH-Owm*W7_wEN*0f{ITA}Huo_1*K}!Wz989b=^@YyaUzte$x-P=`KELo zOv8f0G_Fc?82bu-oP#81f9A}^lXZm9mW<28fGQt;OT;WzA{1H-O~S3(5BU?-J75R$ zRM1lWl16hU#N`+6Y}w2#nT@->o(W@etpG3YY}{1Zx&J{1*aULGI->Mb9dm{k~u09MX%Vh$JBd)1Q3r=H}1 z#|`M%ESWHg9GMIuu1O8f2E!RMqs1Jri3pnaS0XnrO%Ibp zu-Kb;XJA*rAGXad>oH)IEcxME%`M#fp~O^=Rab)z{Vh;PSkg=Gl)=VIQ(K6vvXA(+R#=WYr@t$#XoT4KVCvY@5NJJ>{2JLYsJI{Jkz76^*|$0?(YY z%KEE?JcSU*bf}hYQ>Td2Zt2B>gnUDs$WF=-3MPjjAX0f7(C9NvPS-5!E6cp`&Y_fa zW}>~3nva-MU#`AEBP?V7O=7xq_qvQE$4_INQ&GWwvRiQsRjV^0_>^MI3-zj9#F3?o zL7o)~;`Fs%>MxQ(IB3%Tryt;co5?yA@f*Af`Afrzt%mn`PiJ|~10XKlV8SY|fcUm4 zd$S*DtwG-Np7<}Y`T&!qc+%!m=l$Ro7IDrY0)2PeCOM3WO?-Be zhFN^fM$ySy=|LtnPH2H~*GL9MrLt8rubzeEd~zzb?DMtu0AAqnEB2r5`4suahYrc9 zQVC?CKf+^F3LcFn#1HAJ{Ym5eIp-?$p)3;b&s`X#SAe+#@B+n0@iVmy-u$b`)fS(d zPe6}kzs99_=h>!_hP%kl84?})bcJ0Ggn?G-xL?lt3$`xtDz*kMP7m`X&p1LtylEUA znp}BiL734C&OD)5TF|bgBxFk9Lt@toeZCidAX#!a9DlE)dpVs~vnu_ua`A)<}emebqIzZYp?j zvVUjefzwSbA5`$IGk?;Y0Nu*1r#Px!5@omlrk4HzJ`~Nnq@2;~gwKmd$Q}KnQR~<& z+n~(ndVjWPjb=z=| zzQj)&_MCCl&bUibL6Adn!mTJzl@FlM9{+5+L$6X}oQVlzvfYP|e#41gtVI#mMGm$5 zT{mDcOQFt~SNH75+soJ~`0~JdCkR&j(G7Hwn}l^dG6F`R9u?{|n1VKbAR+@UVxQxj zi9pSe7p9zlzB}$UpL^aJiR3jL_yLMZS*i3y1CWMo9+R>d_To zOYHqSMcvClO3k|4Y0zlOvsH=nTAI9z6hF52dgHlibV6P(g_xTGG=X*82xz{39STVev;(+MiC?8X6e6P7@-H>=p# zr9%4QVl#lEeB(ToRp)R^WtkfYI;zh0YWXg5V(<-H<(ArXeZ1`JtJYu)W#ghAn`~MJ zsLC@F%iuLbuDn>WXq&O+*L^ZZw?!7ophfRQSJ|O>LQ@I&h?R5QtauQRlJj*){DFu1 z0;L2V8?>zG)!!n*|0GWCg+2q@33o~`@I0qOWxZdFQis1mY4=!1M!35&N@5Uv1RNVusrd?QQ{$kdD%GZ4r zI<}ZLEg`XBPPn!$zPYuqQpvimq5Pn2{A+`G^uS6Gi>cva2^ip*dp}K}KQC19xn!?? z_5EgLgQcxWnJei$nkA#c%_%3%2N8#_YjYi%xB{ zQPZweQoLBV1vB~lqzxQ(X{+vV3iSBvd!M zi|P%oF4V{ql$*>kc^vVN3H-;t{C#NRDFvsFd|^h<^r78!^?V-}kMUaaH&o=)g=L|~ z_a~U3**|Kx5M{*_O|F)JSaeS?o$F8yv|VismRMogQp>{i{2&1$R68A9Jy&=IoxEk! zq_AxFn=D;)uKN#`j`IxN(^k|0!G(Ag3x|25x^2O|ri_J{0UkU@@ z&(BOeh`9~sOCmg&#Es{ey2aUbWC&k45mF0ogzKe=PjSs;Am*uAv`Cw(1Z%MHAYc;< z4_Ljbc}xx}jgOGCWZ6DfNVkCa0Pf5P&m?8}^h?9XR`Z9!8ov>LGZyg+Ul{nL-`WvP z1=_aTqTKQav~vBh7e8Wf}$aLY$r$S4C~*P`y_(s+F5OF63FoiBkY)g#-uPDyzo zyI|^s@0`ome{JEuME*RlVnHyC#z8_x8DwKv6|sA)KEEXZZBG|wHnPd^d*qucn58U# z2`&$hFTH*Ll^=kxJTfFbL)=Uf8`uHbj%{s=oRKUERFh`Cbh^3i5SI`zMT_kF(UD-% zczQ@PkHiXG>F(q^v4gkJ&0i;_@7OPC78}Y3vJJ735`kvDig=rNMp(c+H+fmWR@21_ zeHc)3x%xU_#AmR7H&JxZDuWZ5CC)K%eh3&Jz`TK&1z+Q$W}Ul?p&o8o~@HlU!p zj{D|?%qD?;b2L)Gqc;$(D~@ixMOQB7?R;5 zu2G_Fk<`GK9Fk4z-HuW&2zDX8t8hqe-1);47lI7IP+j~|OLVTLa;yRYu$g1#^hMgU zwuLR?2WMbxoG<{c^6mooaVa2ofF1IY-g8%F&2qE&nS|QU%Y~d0h6h>l-v#`eatqFU zh&54{Fo?*DnX0J|q6vF^$V8fX34J%e#eRrI%hh2U!~e#ma9QnX+b{Dr?E~LA|&p z1K6`QuJ8A@FUt&{f@tM0d2GijZnQOZs`clLmMh*QyDc-^bB)JhSC5JFmt~r@UHl3j z>_9FQ``PbFGOHF%f6B*{Z<0tc-QE zMm=BWT`f}RBl&Xkz=DNAB5JxcQ8?~=Rq{+6Az7biPUo7s=a*SU8kbpr{A1NKLFUmz zlWsEwmu{hAvxOO3ZfOoX;s8BRm(`dtR3g;5PD08TwP8l{o7WT2$exlvAoW{5LygEjz){dtF6n-^V9d|?=4`qF8y(%@7Z`4hA`RuzQFBO8nYb%6oYH8d``Rrj{xbM?_!BQML^t5!Q zu*pl}7u|C{SdQ2A&2PgD5>9V-%vqQ^iWzVWoXBvBY53yEf295}dG%XYjOYenu+EpinhFvW-?S>NuAa!GLqS z|1CpU0hR{b-Z^eh?Py4t?s#QK`ZmL_FkECW;;w;{@}o8wNnJ>J<5B|F9H-A|ku$#ikOJbDe zUoYg7_;xKe+PzJ9HOogTdEJj?OJ6uEoF7qNC6Kpl*(@sZRa=RlHM7kvjDxf1j7zL% z<2g+Y2vJl7?B1>F0y#gDhW;QhSh4qfqZ$Y+?Wm=&R z6SGb3V-u40d9hYBde7kq#n^CTOezZ=U9Jg>BN!h6(M~Sq&f1l zR=WmuhLUsS+UdNlw;I-Pnue={ATt7C6PoYYANyeki0syiI^EmN$z43#^NwFi_~dCi zPQ$#b$wl>_HBP7*3BA;(ee*h>>BO$(%b42%0s&tZC%wWvW1fUm3A1iw=KurGUSdJS zay-3M2h5A5KuKH_>x{k6i*T5RB_}CLAW0kW^8l&E*|+1HXuRl6$PP_uVkjfaYO~#U ze3AxYBqrT^iI<0playlacY8!_H9%)M=a!0Ni#JK`vj4r4_jnn0k6x2*mCtsZ5r|dj$Y`9lJda`6@N3 z($6agSS4u1vb0O$2hNZYb&=O@!QpTO&8`I1HF!O4O)zdf!HMHt+u_hqz_lx)-(hw{ z278W__DrzfSB1{Bx>XOQLrEs}>il^67YnP!WW*&W=8cO|yo>-#`%Sy4;JTnU8_NsWYN`=PBX z8f*BPw>GACM@;X-W$YV6c&v;;mYf%CVUE}@FL_K{9!fux6c4(tQ!xz+)O88FSn2ft zK5#0|LcLQEC{OAFE}ufGmG-ODLi4W-H9kA)F#NR#|30BAE}<@vU61XOk7k~6QI7Ov z!iD=P5JMSqzkLk`v+3tY^j zn5e_yzDXgwB|-Jq86sA>p6^jIiF5gC8~~MzXUbx2vF`j9I+Av`XW;p^lBkLU*35pk z%`{bc&@dO|V_ILX)x#99cgN+kp_1}gq#s&w#LJQTq_#RFhh|p<93~P|FR8b=c@N#> zZ6N}_q}E%46`V9EM(i{Q1*NF$8yNd(<~rpz9hU+;IuDSf{x#rga*ub3q5M><9BFVKicqJ}K z%#TmOjB>#Fjy%MNe!*YRo+5A}O2Ud#4d-uhRjYf%xt)1Djg=vw;_J1Mf! zrxYp@p|$K&g>T-Xdvtm(0jSrQ?jBV`<;Nf7r0^dn^YIi#+o6w}?BEuIQuL|SBfk3C zHZ0^XLm%75sMP#8RgHTrkp5@j<5L2@L8VC%)WDeZAxIjTb(^+OmUugCyv9a3T_&it zetb6@GCI{a6-eHPJwu0g^Y&4)XQF2A@&J4zM26G1J*L^cC|-Zoqusw@0Kss%@K{%c zPG#uJJ+5lfEsc!V!k6jxG0FlX6HJ5Pswt*`*_byAr3IfGD2jBc5+g`YeJjuU;Xl@y z0EzLBFoyw&!0Ws7H5!F>8i_Ya&mzxSS>k9HMQfqU9JLxvvc6CDo2kx>yUU)@s{P>7 z5!$nl2f6|WGK5X~NsYbtXQR_nZ?UVzB=eO)G$m!GXF%B!4bPoQpLH|G85#%t++Hmj z_FM?*7)#-$dI~mea9YSK>`c)Csln|`WHK?bwtKGcGDSZg$TI61RSy!IRLh6U^Lx}+ zLwkw?P3r^w(Ujv7X%Su8bX4$K*RSumdSrNeD0sCf03W`UYI+xkqHe}uI<+@ZOtr9r zJ>lB1FT41Z2NQVxd*VwaxeVR!cax76%DZ**oA?rC2pa*^7kV)8%;hAI@O9-%d2xXn z0U)p{IM#dGG4m|PFwTg9gYggk3GP?B1q@(kXWtJNM!sn74%B)x1B*dl8oxQ=GEfI{ zpP>Yr9`Y4m(tYoLRh?o2WRhA1UUsA|eo{f2)prKOjY^;we@H!kbv=P( zLtI%(0)OYWn`f52)5kwNWw}!92ShI)!xkrKm|+o{zrE%U>{g5vDa(7MeJaNeif|Bg;kd5t=6r=>fyDFFe@Md*PnfE8Z!%n#6RKuXAbU9`1t7! zuIv??JM_5;caY2Z$85#cZ2rm4Y#B^zMUVz$?4I8~Z=$N}qN|NA-@$s8=Hvel<5ORo zr@*}2iNbo;wjCoGx-XwU-Yb{yt%%2IyQY}0SAC6kO6*v?UhH3oZdNU{tocTgs{9EX zm<4_>K`d*_ze3!9V=Lrpi4GPyYR|!I4Hx!dQGUDk@PCkzF2=+LiX1G@;ruaip-V)R zBcqxJVgL5szx~Tr*%eJ?BAiXZ{5ek-kSMxBuB!3zb=7A0AFbY1-pIyzj8haDoCWVxDG$RR2Z9PTJ&_C0Mq-pffCy2~lrW z`D$$<2W9~*i>J2oAMZ+_L!KtOZ+VJ@dM3VCA9`c-XVT3vHTJ<*ARPApYoh+yQ>Y>} zr;%Kt{6&RfN9#umlcC5zCwp8vF6QqR`#(P?WV zrBhFCk9|A~vv#w#H!uflQ#Oy*iC(|BPsGgpe6ki+NnAuSOcBIGj}@O2zBNq87v$2w zpQ7~7vH#;8bDG?e1RmwnxYSbuB5zHjNVy#R=nG+sZ7x=Cr5iXga>qaB8Sr>+DgM*n z{=eUa;!}RLWyR4Ve*T{k^FMm~V_KN`ao=XrSi!n`+;_;~ypXNU8h%_09Kk82ylLE= zogBPKcEO~8Vdox6Tt5oV&nYb7y0z|e9sGJ^9?o0gNkJ4xw5BLc>5DF{=aFgsDN!eq0O%K z-*(tvBmc!IXa7#@mxsgurg8p5Y-5Mg&m8|v1^+eZx0G0!rau-J`rkC3_-!GC^at<# zT|oXC<=+G?RE!wwmBpXa)BbN7J75hVGwrv{KaJjhQC!%mm7^!vxlOJdNEY|s+x_vW zoDkNYl9RqXQT&gi{QKCL(cW-TF36gCK~4BGTQUq%;=YARPuFtwT2>0xHtoNOyN5A~C?w4I|P+4@1om-_73d z?{PnSKl<+PkMB6X_dTBfZkU<-UiVt-s`EOpYuy{O`C2MNP5IkJqhGmPFHuqa%bk$H zwL20f-90ErhoH&w2uTF#w?e$%-1^fF{*SvwngSx*;~ToK-N7Um+Fg_&V781fuW-2V z;V`%HHMip7*Sn@_oP)Vd zn&c06W_HANl7zRIi(fCirwhJ%U54HTOX(I*ACiybiza4sIPU zjq==X;Cc1U^KkP@Lu$@<9y4%EF4_ry~>d`RYY=^W|Q8BFeeyNkS zNB1hpNJ(YrF9_mQC0%DdqNri#li|RJi6I_7?*xksm3&UOXHK;=6hHn_Ey&Q~##7RV z`SmoRF9K)DsxXBTfMzT&xXL!td~L72y0bGNfwrANs*}z8UKmeRTDs;XRnh?SS=Y8v z8WIsFD{0AL9+lI-G&^9NZBgvR5gE&Bb8f^llB-(jbnAX{0!JKjouP7-FZN>$r~V<^ z+g}fa|57qUTT#50j)LAR<^20AG`Gn?Sq@Gu*k&?af=y{Ynl)tNM^eyfZYpI&QRWvkNgY1buXy#DuQ4|;EPOONnTgk%s?1O39gdjuHY@Jg zosBd)(T9gSu(*v`Q<}312iWN7!{K@x5z}N`9MW!IEJq7y1$l#RsZ^|6S$BSJ3U4+f z_JCz;rdC(@w0fSx@|Qjs2H5N-Uk%{mbergRQc=$ zQ5#G>(c;fywJ}~Cd;j{cZBV3!-@<&4o6f#^t{iQHDG9B_T2H$~hxe}JBSbh=BO_@` zTPKAnXK4uK*36qeEF$XNx?+k zLaV~y@>tbRKxjoaexW}fL$*SVDH_NMh`dzaAumH2d z5^CSYooL;Gg1r2u2p4#;S+H~eGfWjVm_M4^N=Kwwhn}@vEx547BTxO(DAj8^9Ol>ZYf7c zs=LmBXl>MpMHa(%_Q8_5H;Az;KChY6)5W7<^pBPnx3YkPjq%<~2~)`F2u%oh_iGnK zrh;fU-p@?BgJB%{=|}TH4waW-?O?F877{h}7$d&(s#* zkn6*r?YO!^F<8?sjiVz|pb}b`$4sT|m1XW?{D4$H6>vNwP_vHy=o8nCeFWo5NJvnf z>ghAhvGH+UPHnGkbiuhxS!{Tei)`7Ipmge&>GD!|+@|n-(^1?o(xK_x0IEzL8-p3) zi4U{V7g##S=Xt~U8yi>%*ucnxzCiubD`ek#n(8aPJS7j1@(MiK-GFF=^$49>(}R{@ zL&K;U&_|YP#P$kTZ%_(xdYXBczvx`vl=viTTIo@pJXv1cbUu9>64Cck5MVz)!Gn3G z8G-bD^YsPhBJCOdcj}&$(7L&Y9)P)85>Bg=-%Ni%tfm$6D1;z~uybC1ZR0A9IHVC% zY=1jNH)X+mzVXtu+SY;M#6V6n?G5}&h3643lr^WLvnMx|)f;V0>vZ^bK~0J`s^xf= zR~(9Z-Vn0Rygo)FM<_^ zY*yyyhabDl)jE_Wzv=x@;kNJ)Cbsj|-K+ELh$Qx!wA(~uuOprF*7)4-72GX(m6cwN z+CiK?ZCEWVS(3}oYSPxzC}_~df(s{uUOgXVUSB5X;C|6uD_Nf;yV(=P3h0=IdRn`o zFEDesnl!40%Be#|GfRFu?de5&XynDqWkORQTrEwrFO6i&eU$?>f&)p!!uGTDy2NXI zFkkV`THvs{PjOs$CM;df67#hKzU_kWqpcB$Z%lmG)h2Yi97x%DqjIpnSq!yIu~Z&J zhwLLQvpk}O__#zsrB1rzoFWE}pUcXkSDS6uHP^>axb<6oQXCo=Isz4Qd5yjI+5)ig zN!)bCYzEcwuF*1DrZ|>Fv+MJ}7O39=2Q7yhIz-lgT3#2a(VqH?75noFq!BuX-b_)+a6q@67EJ7ig#+FLHCGyeB9|I1z4WI2xKL#B%CSN`s%~X z6D+yi)#W!7m*z^4HI_$L)hAV0YhXOk;X(s7U!k3^`E=ghH^cH>lWU4QT}Eb9Kk~f& z3zq@k&U^2u`3K^}UVCG;vEMv)e3pCT;>4>Lyr)0a59iobbp=YX@P+#D z$ju*635F|Z`IIf)vso$Scl=fw>wF$DvrWyt2s@YY8LBWEtkfM0#jrpRoA-8;b;x~8 z_}or)%O)sf|m^26AOw|?b&-~<=Of^!~7)rGM=F-*;chMNxO!=n3 zxN%}}7zy6d4^}waBpjh%h)REtVC2%}jtioz11xr5qEMUDZMWqyk)^W_X~Wm)add@$ z>G;U=0u#bKadNd#c*VWff;n46@YSR|Z|jS6k>1NTIPyH&@+Fi`&DmEMyIQS2;LG+^ zw)Yb}cc^=qrk^&mJJVE;SUiHET=-0yvYa;NJ1ATY*q;_e8@=g_iB)Mid0z?Cabw6} zC_H|k?Gedh&j@U`QCUdQ`k_iwgfPN&&F!uJr&o0^+MbGX`_rU5vTeM?6t+L*nzX1l z(5=*IFmDCyk-?A0qy)Nc67UzBNn}f?B>U>QYLhb$%}R>~a(4AL7pfb-@ilcW-)|#< z__GNa%vUW(EeSq@$j?oSd1z(Ta-WFct6{oa*nBKw)9$AHRI6{Ad5Xvjv04cc1f|2M z0O2um1iIVVVXRj_EJoBm=3g?Bza1QqtO7Oh=APj-Z0Wlp5|JQasW)y6>M_@_sb%XwZ}s(sHh7Qb>omdEhRfIkgDgx3QsfRWF7Ct_1T?WDAZ zoA1;coa~NL0Ck$jbx%uQRApWN<%-GBPXRDR_@?D#j*qzOIg<#?D~0dip`DZD-jbFV z5NgWffDP6uU!)#v5jVQ-MD|31Ld!(fM|{pTbhu67bJ``1dn*#o_9rh7I899~KASXq z#MT)yXR6XbScz`+)qYhmYxmk{fD2hKO|r&vacUR65}o~~LdCP!^!Su6DLl?=k^+-y zcRZ<7DVX(O$^Qtp2d)yUDfZessO9(Q+t36wILxDR_uSJ#sd2pD3^3ra2fvF3EG}1Q zf66&;MxY}tJVo5at%hB%EL~!E_a%v8)H|6|9MlorGtuL=6R+_?||o+U<~jGR7~k3IR_{D zaO)t;>tc&szy3x2XnvBUCQQQjyyM#Z!Irzp5mBdvngrkT^&c#;b6mP5FFrIr(bUw+ zHVPnoA%^MJ!AfJ5rXlY6&Cd=q1(Op`IeTZ0PiF@Veae?k696zQ%{tz_ap+l#lkM4g zU?KH-dSWJgql(yy0EZBe2m(sf+M|{+A)6eCC!G%?&^YC=Sb*qB@U0jdJY&AxWP~qB zA2wg%)bqmB112T8e8j?)HGOZvd3Vno2sVFhr5*!t$H~&;Cl&F0VIBi2!<91Kck5*zgM+;(jV+yp#BBu0vl0+P0rF9G}4ZF*kxJX}<9y|8mp#(uwT6p@3 zgwuRZs$<%tWdqEjrpm9O36`BCcy z%0l375;>mY;ZbO z4!Fs#k))++uF%4CzvaOw~iDb5_0)bvo5a>N4-b zmxim8%{sDy=5yadsJQnYhx8?h=+4t#>HpevA_&~1kWFH zO)od?GbPd`Fp#p}efB*D@NS%7a9tC3g$5^aQ&~yr;o1Riqshuzmed(}503_O;Jp%9 zoB6hF^jFD&c0yp5`Uhu$v2`Ef^O0vAh?|hJ%qxu&K!B+<6)1JjC^X-sqau_5D}wxf zR@hd&2%BdPNtA29pXcbhn+5B~(5^Ikn2*3z?bJYuFFIDgn+}Ug*fv}?YpFnyS}-poOG(mw^ub^yZu=yf!&Az#Ax12>t1es$Y3>+y#hSM+5G|C#&zV@r~H0|w_Y>^?)- zv(#!@Re3I>$%^`fz!VaUTQ?XrZm}CTL>u{>$hey$gIkymTC2hwrnr%fr-Vupd=ZCR z;+HNUqO132s%>9oFU;HW?bRmuoe5h@xYLS&6Dy{Jyc#roo8k~D>%FrTwn5(+97n!C ziA~nGNFsH%_GsOGsI$dx^VUjVD$s^xCyIndt!#$bdgEw&>Sd4Tj)~1|WZw0Wy|r9A zUjic)YH&C%&`F*>H_biAYj29J#oMC)RA^}cV`ID|M*6}Cz?7`VUSHk_;LYARJb=hYm|V>rI2fm3POA5GNYjM#d4 zi`IQp;mRg(f_)sjzUH>??(_P^aVnKjGW&)Fv7S6*xoNxvg*MwV2bc-v9`dd`@-u#x z95ndHFlD-Pk}&@|1X@X0bm6nVWo9*0P|dg98t%RpE`eg!-za16N1QUKw|Kd`Uz{E7%RT2{`KKv* zLw;M#DR0(!!uSn0=D%lR|vv2;|uYa z7bpzmTWG}D#G;cV-}=<%?~r@&HR2hhaQ z29NOg>!|Z~Wrq2)x3uSlj1{&Bn!QaTkV#ynIeNUfMIz1X^!hzMGWfU+eW4C7#r|z@ z+u(LxDE~^eNaxZ^ijOz>c%hOPk;tu)k&2#_+PS3t*oz1P^q{lCC?WE>_x9MoUOEcSA$RXPG z&pPj46EKlFZ7l(uR)xx)00=BiIbt{tp4bUk?kzY5?sl3h6#PL5@v7}%u^I`$3AhtxA&F9OzQ zcW3j4M0*>bwM$%rhKNG|45Pu^rUkVskIK$|^ntz#LvoXp=ZSQ^Z%Gh0|>236KP(ii-OC8G(~hDQXj zq9peo1`zgiqV&M|tpl;o)xx4Pp7_UQ8r36W&$x8*E|CYotz})gJ+Geq@yy$Pprg;7 z$ZW05l9{LYy(bL`2vrleuoAADb zC4Wx{`C`{vvnAFoPUpoat&+x)oAM@SW+m%4{)ZqIuG8pxgxjbEP$UOp7vK#Fm2-~z z5f9{yB7D!!4L`oZHmj0-&W5W^&8bnP{(vLepv`e<4t-5&W#ROVgSmx8uW*VFOt3p{ z;@k7wbtJa_#rS&r(bgQ7IV3LXOMt(KEdKV(A1h1_GvhpB)93G3<2j}i=tZ6{ej`S? z`Q5DW*)rhBPciR}<6qENR8f{JqhDBt$D0-QLd;M%gOVUQ#baTN@Cz_ z`?HA>;|fF5PQ_?~LS_7G)ZswlPtn@CKzo;}CoR*@hpSI&W1=)ldZH*oB}w?>`X}Jj z3}c0(4Y%jQPyh(8v7QeYAzV#@hG$rDXs)#?Ey0=-BKMz@`jB#ed-AN*hs1Um{bn(c z7%^eG!BVWAs4cm2J71?b(NdUj@t``DFtNl zuHOF6F#wzvl?8N}`=BZK55@Zj{X&COKICDlSwa?8=f|65`;7FY9GD}drV3Q`j${%v znxMT`7$bZ1qq6q7pr-&#<*Qu?uTn@IR*d6iv74^hZvsVW^hC0BHhm$s5cYqamTP}_ zkN5%{aslpA9U{St;hKR8lh#ST!FhQ7u7KG)&{;qv#mpZM|BtvOTp8j;=*WE9)6T2NWpx1yPX>TA8| zx#`34G0zGR0bc@;^FPKB07Xv51cwkcXXxG&boCk}6h+J`5oXd<`_&2mCa2?TIUasw2Kdk>ov5@M6>wr^~y(b<_x!x71 zsAdqs3DpE^b`EDfjwou8G&~n&$G`T*fhIc+Yb>-(CgFyaCO6N<{50~&r)TLp_4Qk< zjhRo=y?VZA){$1)Q5UbKdHl8gm$AhGUhR9^=Z=b+vh%}7N&$9ZZYiR`+73;YFPS5KzS76UH8C$n1_FAV839Duq&@I z8Gj8Q{M5$a(@Pb7>hg`9@7E~V&*$(0X5#NV|Bs>jdp7>n3;1tm<2!RaL1}C zp+mf6b9*jec3JqOAH8JmYT@E?*qFp*aV~yIS)Nq!dxp_CcMlc(uS5H}f&X;nM~C?q z^Atnl(|_~bIUEAKvKgAp)<50i7r#8dPJg@Y8foW$^WFdZhYZ$}UM9D61e39a_8?e- zS%idG^0-1mLn8uJ5*1TtuRNfUGsmgW{61OERAIaKP}K9#Hpd0P+bc8=G7=M}Swk57 zWWM1bp+^XFBJ)KQab&WW!H*vxg%6P|*&0!hV-c>GS=9dqt^JRT63{=>0#@19W0Ajb z#?HRMK-gX{>hp@prlbt*2BP&U4delVSwLHE+NbDRyu*H^R@dqPQ=tf(xksZQ&_cAG z$cx`2{_5JHPgk6z0zIcD(tr*=7D@2h>hvAj~{va^r~M1 z=XK`S&n{ZRk*p_34h0n@Vucw&fXD|X+MKV}!aMZt+4Tcxc6U~ye$5b;ak)u= z3ZrDB$Y@e~-5G9tqfI~Wt;`aEc8QK8b4tEmcH-A>cZU?^Ut5ub>2U&^<_(8OM*Q!; zE%?o#O)+IlD|VWRQN0bVvugGY_0BR~onOBy1bmFaDo|pbst3|X*F8AN0Mb?~^?HlSZV8eEWlxg8AtlIcdwWv3j8-sh7??zbdP*D`{def12;_$Ag=Zk&&g|&KT>>7tS}7DdAdTq*)h!aE}(h z3=s9_6!HoRvGaWAQ0IA6NMg6-k>n=j)pVb8gI3V;-!yc}J6B&8fV^b=z(~yX!b2qXc zD_Nbk#Qteu|LqE#%-P0|DS9yAI0tb0UH%~wu>Fdn9;;RR+JnVS79O=DH4%z|#XxaCyvmf;w=RVc$4w^% zB0(zSYSvZLKa4_I0JS%n%bWTe4NAB8!nvr%BEjCC$HX{;b~|(RkHjS1a+{8ADVmiY zwjE+wty>MFJ1vbSG_-h#Ut2b!jnTi`fg_ik*<5K%BWa-WO!*nQrWmwA+`l{-@u$aAYbL**(mm2c;Lesc>2jW)>Mpip>m%A4pAs*N6Z=^R8Ud$Zt&WS8# z8Mw41WW0Q_^v%#d>SIhnK{h65_lLzUGe}MdqNA2fazOgcm)zl%fy3ddshb0jPC|j* z&gv^{v=rQ z$Cv1nIJ6=x724xU##Yyh%3CJ%IrnaPEGHoPi70^B@X8tszelT=kB*=Cj<@<9LE9IF zHtChWmdmTq;JSbRtP1KU z=HY<_@~Wq&BR#$IP9+Wu6{Ng-hrR|@7Xq>;$=RJ-L zs#zDO`KP-0tBMJFBdkwrfl6XnP_j*@-a=j8I5x4GIANaBNZfr(x$ifJ0Ok4U@(f@8 zGE=p%0f{3`=@A(W%T7akyTY8~(xus$p^kYS0cUFGtwskyE021kQY=TT6VZV*X$@NV zWaruIsb$tTaM6FV(t*PfSbA8v~1eWc)9|DCGA#2L6kdNtfX4M#Hj*jZ%$ z2WfYjo_v9cw$>dXE8aL$$7b!;y3ntkos{q*qn5|hwe=b+&;)ZmRaPuf<7bfAkcvT` zXDc=0J28Yeq1O;eCttsPQ%Gu@@g(1wE$@FRohfb((cQi8G*LoBi{7Mvh7U$ot`9d_ zy6Wl{Y8NwX(XD;|-wcnaOu5ZHn7UpFv^kLhdlCBaBY)F58Cw|Si+iQ{G1{Rg1J_Xk z?zHWeb}HqNK+dcEl_`w|2h`9OYG)`(03WsaUhYt>RS~^9de-Q&84mD73~6qt>jOvn zO2BF46%mt`_?Y4Z%4hZ&-1>JX#>US5=XJm~d%x92GT)mp+A&snK%24p^nQ3fN#SZG zE!(VJ6VV?`dmYuBokVHGLFy)32Te_93%7M?selr<3cC%9Yk@}fpq@bILP{nl~v5K+2pVodtkH86~5oc^R^%x&3*QuRjQAslW->jenE&<*y@)`7D zdVk7MH^Mx4sWL36(b`-3#KwJb?s`V92pfZ6n<%(AN!VSz3Gq-Q?ChD#;np0P`a#SE zcML37Ey}pD2C|(ZZoWFORiVP88y=f@XK+vZA^k8e@~Ti*C^g7QKupZQ#A|s>euw|; z(O>QHf7(hVU@YC5KIoLDeKM#A4)*)x7=ZPw5K}O=<$4c!8#tKUcD9Ol#GphU;`PCA zX-R;8eS>eQQcbpZ+E*9HRkd1z;0k6Vf$eOWj%?FKPGKXoc~%WLFZs9j5);SkateeP zx*n!K?$HcANYK(8P&04wnY;5Ale>?-*Bxs}LxYyWdoH9_u}3po0)aJ^uUXu1X^lDx z^7AQuC1J^Z_kK7x4x{F?*)FzgdE5=}k?cF--pWkkDmQ;ISGwh1b<|tv ze+6fk^f_DO6+tVPCpo5?kR%5PQrL?0-~6bbe}d(2JhSkVZ=e!hgY=p!NqA6Zk294z zDqx3601sKm0xBMk=#M>4a}iEQv^VI|AWVSf@`XybDhJtTx@FF z0AXafh(nfh@cF;HxulwaWA(5%srimH2pN9qpo9fmJ5Dr5h6hRKxuusyE#emF6f#u9 zjTj4cxE=MVm8ZF2Y6X}Be{}?&8|oc_-TD*Q{|Lgn+{ex-pJS0 zk4JSa@soNC7;XMvUBm%~U93U1^;8MW>GtLH(DUP1^u;qwn0VTmIR%>h$H-U#UzmW* zDJ#Wdep7n6n4z?H+gKbMo@he*{lwnb43;`qf;u51>QO4a`g*Gp$VRW?WMpP8$LPX# zzq=JfgDcGTu)-t?Q<^gdDz^6d=b1g!814>DY-WCTuF#e?jDSnsHpzWy)lk&q^ z%WMC#XtvygCC1|GGqn!FEf0DBS5f!R)Tk+@1t9c2*F}9#4}-lVei)zZZao-=*EB8# z8rsKr#0f?+c79X*sX*;|d&3M0%AJ z^rKtifs^g0!k1dDXjx>OKxbE>b981V^OOxI$g=+Y9O>AcHFZ>C0T!Pj9k^hlkVCh&h$L zoLCpQL<(hY5v@JLxbg3xp{PdNe)VWFd-$W$lRHB3dAa-|iakBKD%hC3R&M!TXB}79 z$D4zOJ})u&`6bB!;icKP%MNsz#&aO39Xvr8con7*UzDm2x8V{QLAs6c)DOJ0MW?`2 zjc4749ogCm2-;;Eo#OCP;0OVjO7i#X1nK^+QfQPXG(jRGLn74t_n9Vo&nvgX@GMJh zmaMPhBpSH{^cl8jph&z70hh>LebGzD)+kv}T`Sf1c+c%b78gsC>ZA*R9O-8{w9iQ{8RPJQu} z+LtuV55d7AMY?i4XM3Grfi526<@d@pGaf7GHh(dhSm(Sn4UFpa<>_59mMO~!H@)lT=LGQ^ok<;Fg(7)Ja=J@ zd+}?xIdqENxeB8R%nZR2^<>OyJ@0Hiy^U`F`#${{h3Ej$=dg#78>f1ZLrhNTBYB7aZvKbPR|N&aIi z`u)9={}7n}cj|&(xDb`hB_+$uSvWA@e!bfSYp;BD(F0VNRvdrZr>^upG>kZAG6FK|vU!Hw>UXMPXe5ArKFk=LDw)(Ei@ zxYU__jaG!^Z3yVUld`AY4#W9l75{jJKg69Lz)1Hvo;T7{ve6wfyy!EjysFi5>13|2M5M+7tne%|+Au z=du0u&C5Om3R}3z%IUw!$YOE<8oS7e{nugU&k(&W5K!2YyE#MVfBoQ}^Kg0f_ay(E zrN1Zn4^jJjC;t?aztzQ0efGDy_*-54p(Osr=KO8={+Nuv)y3cH;-`}MdG)t<@rUC0 z|D|`~MOqvGKfM5c?%=<=`r8Qpf48g4g4*)6a+j8Rq7K)%oBzS$?Y;y@%nE0EI{!Ug z@iVskZc4xW^~zMMee%y0v4a1D}_lO#y@YN5r8I( zGU1a@hx_AS6SI(FH>xXRW@QcB)coSz5^hXD%tVR>u6JC$Q-cB?Pqh-2@LxO-158Sk zm6g@q2F50af{x}~saDVs31@ip)jNXnx48Ab-}ahV9pvEURR#Lz(T^4CQk>zvUha)K z6N*^?NzQ<2^^(P$3%t$_CL)uQH%DT(p)ZRd(|qtt z$kOF9Bv*9}ZuvDG$xr4nk#eG-!eys}kuIQFg<=A~xfcSv=eDqIJaPvHtAc-`6rkRC z2ThfnsZy0Cij0h;1`|9AA~~3ON1}NQ%r>=YnDlv_)7JFZ&<^91kH?bZ`V6zHj;V1w z0A4BMLjHarl%iAUHxR0yH|{`QJ5rKMr~5D<*leO=8DxegnKA17-tJF~22Sb2?>Lp= z54uLTT|1pD>Uh~-%x42vYhf`=IyRC;Ns%6mA-e^q4+PGF>w%9IhJ5r!eDRh_BMArg z2LsuCuj8^J^|TEl3f^dQNNmsH*32Up-aAr!GI!hDiE92z75%UO(Ffzkexl>U=8O)1 zt}7U;F=5I}XFT^fa6F$gt;Q;lrjQAm+bB%E7y<}yMm9Ka z^k&DIa9)=7BbFgbK;yPYz6fd`ytkO3W9+u&UxVSK3RviKWAn)rW1^MYD{`7Ez?!_8 zq@f`QXH%o?;P*`+8>_$uvz}RaTVxHSh(prB{`{F7a#TR~h$>sI+11L;s{sV`;pV zHZMdn<$_e&ev{YbA>yF70HhNbJ0s1pMaYy%KBs0*iJECli2lp-Z;>Ada@sE5}FS1uLiioc06cy!?f?V zhqwC!)Y&nOShjXIop-O(pqZIKdJ_XnRL0L73)H5va^p9M7=0DiNREnt!~-Lv3ed0RrJi}4AEsv2D~*l{p!|(;1wV2MIDkw+Nl5^h4C|*St7*Q<*qBH#An;L` zrOmA3uG`TrThpS`1zIz1Id6ecAiuC*2GAAx7y{U4BUBLczg`)cal$ON-0sk9rRBkK z9?*|AOwojB=Diby!ixm{&{C{yMWyx%a~3fv-Vw0|HI?fvmYsy6p9h@byU%-4;@MA> zVxtiuca&b#S5Qhp18O{lg;Z2&B-rygIBq0y#W9m}Fy6T7=BgQ*7B0{g+M|EaYIMb& zMXKBn<`2!u>Q!(asDy8=&ldRDK76}yPdt`8Leg4tp9yZaVgwIaeeA5JI9|fj$=lFf z*5bMHx!=$Xa_SB`ZhfgVQOuP+T3#RRd6$FB>+O1){#=pUw|>+O+w*pnB=gl_7LeD4 zkLc(5rm&tALqtLQ0hTo!{ZS=JarLIC}E1vg00bs*e9|KJt zpG*g^2}9SR2bGpA4)aZ+8;FT5@&ku%@P*U_84-Cnj&rhxL6ci~k#^d~3~-F==?OwS z*}4$A0_Zxbr$^m0P41P8t8!4}XUq0MumoxnXrSpax?dZ0Q|cT=F7K`*u?R+Tg^RJN zrAD`2oX=c;pXF;pLL;{1Rzpa^%gv$~9Ue$bCFUcGZ3SsorH9Trw4Sj-VtL5Uj5g9a zBpnqM4$!*H%x^In7kCHpe>y4udewaki=!(Pf>0ncKY6!T;oZ)CFZC<>H2YiEH--IG z&z@!DwD7vqlK~2xa%Wj$lCy(zbcNaIYf&F$`g+6@E)(Yd25COx9Xmda@_RW4$+!t) z+f^o#=I8j2B)NaQc#*w4(~&jge@#ry^4QqZ$LZi^h3}$u7~(ApM4=02Q2<8s)1C~Z zh|>A6k9fKq-M92di}!BPd94*Oa6<+w)IYEHw9q-OOOu}5QkPiF9rNq=Jh0Z6YHdgZ zG-}K|t|Qv1e!)Lwspe?@sOM-vUUy>}O0yTieflb%db+x^aq?xWSBC`%Vz?up@8K+o zD0V?U3)cYJ??!0Uz?u?#2ec==hI3PCI?RE{V8PC9DYOL=8Huo)PSEkM#g3&@)*2p|h8UYwsT_pqfoU1dc} zwxz$N{pz*Td_-;P(Kg3DR-~_G;<3(R1#4+u+eaxSjM<09d3FN^KteS)uH@3`kw6YU zqCJ@xpRPcLY0q4q5`mvbnHaF0B({*_ron@J%S8W}` z1@&Hp7na6Oj27Jl^sOt@#g@;gPTd2!wm{&v$DvL6v_LyTP3q(^mOg@s%j*-0`jrWL z>YKBo^#^YXS9Oon(q`D-XW2}YAU>mNCBn1fwA>HIbe91|cxY%0>jL_zbnZC9ZM$KP zwK}-ZYph0ObxIFUC7(bE9DjVhG6bs>D_54$fng4W{Y7E~P}heP>JDj8j}0D5z0dEw zE?NgFg#2VMyoY)KvRYT06n~bY{)r)R22N6}??FtHMs;%E9kI3@8SCL#XT+ISikaI! zWGSot&eHbkK%a}qIe-64b(1tU7M`ayr$Ugk=)jqvF{*)iFh;|V1<9G^wr|9NU2A=I z`zxqqM@XNeznfV1^M`29<)qM~1U#%Y;Tqm#o|g_SI=2cjWpovl;;c%Mk$_WSe}^D_ z0}ft!Q*6-vQ0Vm%pLsx7s}yBy&@Lsc#KN+S(Cpr3nBmIDD~q#-pF~9R9?X6gD(Ee3 zHr=HnNH?6UaVz;$2iHtEIGo8~&Du>uCjGD&*)Sz<<0g}3JBC9_KUFNZU%Y*lC}_oq z3enBXvLu(+J>rfR^qRao0Yl&Kd0EXN(i47Kx?S>6rfTHWY+m|4)3@`={ zrYzIyr-!9$`HnW%?~V$X#;0U6;P*PH-yB(kr6_FmmD=upI&R*%Ujl9ZzS*CskXAmFWvNc@| zaX>HR$=sE>A1#XiAK5iCI?Mx|&mR4cmXPayc=(L)FJaWz_4DVWQOmj!J1rFYReF!l z9_S2X-(G&7N7TcbGF59jYSh6vGM1D@#`d94!*^-E;G8`_N4VB|7&FSND zACVTHx_>6+R(3jmVXmMeVrnx3M?jg=V)LOcF?K}fH3cgghv+si|N34KH6|`Cu8+%M z(V5&is_m2qu3o*7YxtZ)s@yg0QyofC|14_HX?s3p@KHWY@+8mp!t?W5ZLhS0Yh0iA zh02a~AR0AEm%7fl&-d)ir6aJCT_75Iyq+n?w^k6TRU~a1s~S(AR@W-8= zEjGPI>3!y$VaZnps1tOl)f>POQGxUk4kU{LhbvF#oz~1aIGRhN)-p@%j)>1>3uuq8 z3a*3o3c-!3)@6bMT6NI4h?Z;Ag6gTh=O^j@oXM-hBgL-pQH9IUrVoMkZri|Kn)lps z#vwm0-qd-(XPj<%xoUTg+5rNYw4oOMQQ7~*!K~*-yO-6rgPE8#ZYmx?;EA~ zUPb2#0){P2-(?xIUa+U_Wl|j1m*9^S4@+O1GI^{=PT*fHE3hy)1YA-1Y{eL5X-k!B zr=>D{j-WZ`2~oVWet4>a)Jq-m@zcq&)R3sZaGYGo2mNb|#e$2Bv#>B`@lhG0MuX;0 z@UGpon&;*(RlZKg8#7rkR-Kh4alb-t3hdcm711i7LgL*L=GBc64HBf^&L?e^F1FXkljR>{WR?H3You-l1c%mMH zqZ9t&i8P;64yK?J>iuq}ux=olIa8%5iE~*StBiy0fh_=Xa(kI|-n5W}e!Rc|H+Gzm z9X$Ri{iK%b=H$>l>__xU(r&eLO3fi9Vb{#8@OEA@SxHAj-L{ENjcP##tv=0{u62Q; zjJbyejyD~Z;4At#Banhj*P5c(;6lxey@1iT&gO_U_Jc+T9OexeJ5##9(_b`C=>}m| zZx)Qr(LbCV^ON7n1&_3}L)ad8+GCz2PbYk-#fx#R$< zeQ~8FBrsvb3%+qxGUnn7R4PVkEPB*py*o4o;cdCmu5z}zs_A@3p}=F~p4FM6f_Pyn zS8cTY?l5lXCtJpY~;)Qw;`z;nE2BJdrbIdowZt7_N>=(@I21@3MhIl zo4$5ZD1Evk2>f1(jX#`Iuyq`nXC-OE&pjf|q}wtnS778A60N;aLie;%c2j>Z>=g0C z5hSS4Z~r6aR_M75?*{4}d5Nt=S?gmS=V#H1(d-OY;Fi*pf<8@zCH(U98|5NTu|Ip{ zV`SE%4p1u|%^LvCm$GUp!CdHm=>!P6>0H|3Ed?zkw%q|xDyf?VC_7jq>j03=C=Ff@`9pqnWjGl;3bGk*?i~(BlKNVkg{DQW4iW(XiZm^(1;%L_KhY{#Y#Sh;AwD#-Lb3`0s zsb{fNQLpz39s!X6+kdhNX9&4<9Z>6FP*S!;t15+iqicAxj(@=7{LOqr-6ht2={DS{ zwScG6&YNFC!=f_}rmNDtNzEKa&u)LlB7o|8uyf1V2SKvmpyY~yLZJA@~)WE>xsH{O$a4sso7-weoh^_w zqPFe)>{A`(GiCC|yhLp+hvBg1>X=P~{Mg|)N@B_j6D{6LscO>B5Xn|6!TW_u2Oh)i zIWK|u6Q{ZgtTF76>A@!1Z7V45XVPc_y*pU&lxyda*P-k7?0VE{U!F;GgQLJmOAy+N za=nhW&uLz_S?5#$D#7I8H~h+P2UjQwnxzo+)fNv|GNkqCu-54Y+B_QIV4GKx8 zgHZ-vOriGEeZvCGsVCO5*piiCSC54&5rsM*Zxg@pm!`rJ<047Z zl=w6$+fpo;Tl)bAJS_Wq%u2!PYV-Mu@khS&SnNDT&{BA!kjw?cajS@rz;0mCi2`Hp z5}d*`8VkVcc!bPd10cNnGv#9N{$X>|S?&$Xq6dRBvgvh9Z$_(=!9^0`D*^fPH}wD)SC=aKwpl)XycJZ^Mm?Q%-@PSHj^58Rdlu+) zMT~6{S6+L9_y4f>mQhuDXxbc5ug5s+?Fkd|(c?(PQZTr|>1 z_o6%Y$^E?VH{SQV@8{Y7_OCt08V*@&u(+=CI?ox$JmxVwZ?A8s>@3ee_Uh$FUB3w3 z15+_BSGUk_Cp}BIa#G!e@IMoWQX`BLZ!Pqi=fMfI3sV3_sZHIsLlyf2x?7cH7kT4Xf5BBl6L zb)|73jf@1~S@EKilg7=BR)KH%I_LK9M$c5X7kLCEchLifzl?+samP?@LrMWQ9;3sk zRb9BbhLE2Z@17)e-#C%0A*!i8^(%hu(Bsem>ir_@ZQ}0NE5a$8^&RG9&*Zy;v_M_C zB_d8yX$mPQ+i;N<_gKFD`T*;V#soaS$MqSm2b0I3xYVQT8-Rj~i+LW4hmq~6TeoW1 z{w~#j0IA~$+u~!R7bm=|e*6>CijJI}FVtmob2OPBtDeyvd%Tk=&h1a6_cvCbIJ zVR*H2JM64jxhhSvTaGe>IL~EHI^QkJx+8&;(qmla>HUzsioI1P$#abY6n~2;J7eR^ zuI5`x9WurF15|_Wq4+Lh0!oA9v(?7ZsW6iS01ZfB_zm4Ct=8`hZ9p=LBFu-EOd=f` z2AL{GIj4wF5UFk(e(hpuh9*JDRj#DjU+Qo zgD>@9B)=P{SqNG!^N#?;d0qL3c_oOsnWgY(=?f;&`c4)CY9583=yr%QEE7UNz-sXI zVyAnd$*l>~VNdM?JGYS|>O&BBKT~A^`huDX$J3_ryjFq%U(N(&*oJzC`!*zk?B~f- zq=Vuq5k`iLmPRGuYz5`1cDdCm+&63bLt;8MXQ~PsPe!E;DuaFZ-42&KCj#}3vt;7g z1Y_;ytEPBN2#3E6ENp#89U8MTIm0Ge*m$z$S(|b=Q17Sbb*rLU$Z|)T2TZ)558VF7 zB(x1^T3Ws7Y|$MOedK_A&2=tN($}xyi#ON z_oeKvr7`=_;F0u*O@fM|jYU`j?RFJs_@sHC3CL_CT`jbjis{%qeOK72(J`;aMv!mhd?>jT2LzYsE(4 zlYI~OkK0hNKtsG$`^wN3Dc0@!Y!K2xIgSBPo9uz(ZU$IMx$b} z8c(MHMS{)u*?3#r10kbrUmlU#pb|$DVsP;7IBaM7K4Oo1J7JB|@VvbJQ$!XD}X6y1)^H3aIBVuDtlGFMQ4CwCJE)G7MC3NXxM>7d|n!+iiQ?LHWg$L7Z zS&9=eL29OX{$m$MWIeCeqXYtE1a*=KYB5l)e8TtYZ`C}X@Aw2fnjt-Pm~*-Cvv;>I zn`i64aE^bBk!Ooj{jDIO6Qj&YaL|*zQ-59im0<<`v};(I^TR@6-X-D6I8};MB|Fn% z)9f1|2D?)kl;>SnDVOul%%AQAc^vn$;(k{`#4({`?6hzUyQjBKxor6J0$DoBq0q@% zoj}+mcLrl=T)#;1r?wdGy>ib{Qove&d`xWP@1^6>B1ib3Qy=;Kw$`K=wxeK3%*dpM z7ZI0A<$7m?C(SfWZNZ=uJxr+9Hbeey#LyfCKN4qGxL$)ocrbx8t6nqk*J~H~XE0r; zWj6ddud4}sMWu(tCqyHYz1j^B#O!Z8U8^Q@=8`tC$=+!*%}@ryr*%EM=iC{Hpfz=( zc*?BYy(V9?ymQfxWR20JX1E_-ow`66K4bb#;Y{CCP7d!qc?dJ0?JodjU{8C1i z`uYWqe@E){#}t%g{RQur*`$&vjeSb}-6SnXCK*v-W@fkkJ=PG;#J{GHyD(b4IbGlL zXMCDD_wccVVpF47piwPfOm}`&XoVJPCwq#9_@hZeXl}W|GJbjsTf$QUBt#Nu$475l z*LB=4Ct#BkuckoDHR(U*ql(X#2>J)cBfZ&FJyG5w#weDJwOXju@aiAO**`W`-H2t> zqH*oJ_t<9F@!H~UKOJA5;gN&$M}XqzG^yiFU%a1|JTM38m8jN*4WLednF-g$=pBkLxKUm;>?#lp@a%P57*(`9>?Qsa=z5IO%Vev`*kHvlD<>qMR&Ti-oXhLa4<@%-1Gzv_z^c}ZlmoL#bKzl2M(0#<)q;SU_%JqC52K+jffqUKu8bPDwk(!gFCBJ zJNY{e{@06A*Pfhogk(8l~h9}^eIUs!*Kx|%K2wDsnu#NF~M7zA{D)D?cVfG z$}pi--JQ9!m3gtFwd288uPPB3CT8VY+FMi)Khpj6V~Y z+vqp#?hYd-ALVQdQ-9sBTxr@RG$lG=El$eAWQY9rHW)G)O|HaVn#AuiTFO(fL>@}-7nP*li#%$WmbF1Hc+k~HE+5UUCl5xiL3?`hr2}wO%RK?uKTUr-?Dxv zffg+$F=&Zrn&XByy1cA`nFvTL*dTeHR`m?S=Zn{?F`m+BE#B$6Xf!r>*4aFIAFEta zxsN@9sA0eH-BMA6^VQQlmz;^29ZvHA<&qb6$AcUD;jocuN=R1ynT;sH}R;(B)wm(#s9A>Id&5FFm;O1F_D9B)bf`r!r zPB9jXmRJKSZP2dWmb(am9!S^OZQD;%a9Yo)108ka50vRs^)~iT#PxAt!IfbSdVHWLaPVsd^5}HY=v-H^JaOv70=iL40D9jW~3YcWgB@na|^BfbDF5-a?gXg3?TbX-pSI57Z3~Oh_KcNB^5THLXA) z+Y~-uld}=cY5u~_#U9F^xSP!QGlnxR9p$6jxoSU|%)#Q0+5%492HE{7WnGuXsoyQc z{O4^@IHgUHWF# zeg>gQrkSfb(edgEjG@r2b=ZEHONupql=?|kb)G8rHk`{)f413r`rXu|6fmd~3XT=p+Fvc3}44g~|> zi9EXj*k%c2M}VM{-)lW?$&F%mzP}_i1?VgcEdckCw*WQ2(_S>1m&$wFf}0!jw4|5@ z8t54-71AlB?!PXW!N1+;z-W% z9eZmLy1?|ri= z;pM3gmQI$5UK^bYF`)M&^a2q6YOd}GpJn&gotGNndIMzt?vtKc zK_lnmMv>lBU;46$IM{k{r8M{M>)82+(>miyzd zsR()a%8f$od`E|CWK;sL;u7sVgu5H6TN0CVqrV=w>iP9}UaQblLHx~AoB?Pbrji%S zp4>Mqq`>%;*TKk0h+m&2D*fF*d_^S@e0@~xYB()0KYKZ+d9v@~I6FCp0aW>W461H? zTB0(u=iDzG!@2#h3J4dD@Fy%W1}A2n7-6_|H9X>G%`boN^4J0IOp99ZM{AV z3*+NrVVUJ=U$(k_78i&dYgXl4M{Z%GnhL6ROu<|AiI}I_%A%0qFVvY_(wk+^F9VQ-QVpfu!DUT zxyqzIyrMmnEZR`rLl)$dQFOfC8zqOyTX5~p?e=9l$KWgEUA=s5s;x)E@(~rmLe=Ux zPL*N+Y8)y>bVFghe$mTor`{MId-Q%Ls^bXPQ*=x{z{=1EB$hy22BX`n(|wKNy_6=` zUVzB>UYT1TPP3auB&2mw2^@Ajr=i{`f7E!E(0Dqj730u|=5Wxa<1|lA%IA2(6afOG z97GyIx@~)p$wCdkEKqK()4%&>q?)JLd2;aVrFdLS9sX;rY8lm{BlOQtz(#8XY%x2b zCJQ&fg-cI`RB$-$Ynx3KCu@{xift8|#V4au&cB+2);2pmdD4}-lb4rgw;p`eB;ajy zx_T*KZT6 z_)|W@Nt2=eEe)WMc-(lMqSRHxtv$B)D3?M z&EMLPuH8skyj!&j_q>I|Ag7a7<}BK`;W7)#J2>ZxKJfD?*e>M-Jr?<4slm_Un`APJ zU~peDW{G3JTP{RQs4cnNZKNpm6Wq`q)wwckjlm{_?_W_u3w#a>#vSrGS3M%HGA>N3 zzP7u?pL1oyczk;T*i-P-bxCO1j+t=_N2D?XP~mcQq%)&Ym*Y4zGGp4j=khN_k~qo> zwtW<<;C%Jpm!YP7kd7>j;}|)3Q!ro`%f-s5dp6x&Ze4#h%`H_Az)uGbwMx{7g8sk*h!y#9JkIK|B|VjyfX zHrI-B#Gr}(UEb~Rn~aw=li7#G(!mE1Tc7l^X*i7>$Ng*P?Yx}T8H*B(?E+4HQ^7L9f(-|Abd+T&%{<#-AC3OhCh?Gap%wPs}a&7FpDC-dr1RXpX-w1Zu+ec-^_NUeN4Q@KxPx zQ#Tm5qfxdkTSK{4K;-B?y^%_|ogP^txgbguFk$Tr>;h036KcW{s zcXW1VO0;r{EoR-?O+zfsc};A*cHlFxJA)e3FidC7iQML6sil=mlccUBF$AxrBVWls zOdjFM?uU)#gvlt}x%M5}u5<;Yh_?XVWO`1nQ>xe!D1Z13`dz#GB_H(3d}u*b<$hvt zbuuR3WS%-J;CWL!=ekqG=D4fgZdV{95ZNxvJ#VBghMBbJyb|)X-(hE}Zrf4;x(`Y9 zpmEi-7cl%H#|wSHvpt?x#It;kPw_|xaxh1ryx+rhYBLBZZY0tl9w@izi-AK={%dvI zbSIbXn?P(4O3iw5YoNaGl6#zl|1mq+faEG#<`QM={7;VCzj_y2@eo|SPYb7rDN71Oj(l8M z<|(q=CMHz68eg)VAyz%3KJp_PS3yMAh&~~A(`8grzHL#XXvHmh=T{&kn8(DcXc^q2 zB%cevyvPI(f8|T&{=sOAJXHGlqGh6>;>6SWq;pYjOOY})uk84k=b@1XD{2F zAXBHp=Tl+)lxqCPg*bWY3K8Qcy~HMWFQmJIBfiXX^AsT8-gZst#{N;eHE~Y5>rGv# z@n2zJqM1BIF)^=)5hs{I4qZxPjiLwK-#_ zx$bMR35be}dt=g@Pr`CwR`RuK@ekRS+t-i?=zXE3l2X!CGoY@VF*X0Dij$sLDAoV6 zGH8{7cq*Ky3!9I4d{H`-$5Y*weLKSL}7v0hE4$MX^f*PhlQXa^5f6$*_ZM=PyyqO7kHr&SM zit6bFuK^XjE;Ph@o*HRv=a)j=Wod#=`sR6l#{gVfMAqLoVX42jYh?ecttIahqZMYD zh+}t~`Ois(`{3`cqO1&K_eb5*0dF{mVJYd$%AeX>j{$yeJeZK5H=pVR<*6x(#s8hU znn<3VJlchUX!B#pN4H0<;qGGvJ{RP)b zn}~-CLypWRbnRx<<~aAyvPzdldSPg&Ko>M7Urh}|k_hwDwjE^DW|9~gVe_cg-JA!f z1f=E}6Tt+7DQ>+xmOikU;v(Oea@+DRAkaEo-Y(iztQtqB$Ll40{Iej5-?pPPo2<6P ze#67hm{;i@kbcYp0VhwG#>rE?d23kL)Af$-(q7U_qS=kYyEt|Iv^sCdo~Gb*daC%C zRN0XX!dcWfL3}r~GT2;Wv$#+wDW1!%G>DRUB9hGK6qA$cIdgImjI$Y!FrZiqg-l|e z-*BO{Q4iVOiZ_$hj(>^f&3*d_MO%`>+gh#02@I0rlh?n}k#DiVdHfTzCzd%CY70Oz zUF4xn4GOjdrlyn8U^4TdmEEQ1`w(qr!qHmW6>nJTv?3L6t}s+`nHgZY{j6?&Ey54k zQo+59;(c;dXI|+v?05VyvoAawd^pYDiejx@2-_G1GP3>^jt6yeDb9=id1)x4sJKUi zPU~?pvJ8oAckAU9TcsY0hHPu@R>$0&?UOYh zkz!ha)DM+b$Qw0QHTXpeBFOWr5UbnlB83EQrxbxfUTNhocs~>9!xOQP{lmIe9f1EdKz_47|*5c#RMo^I(G@&uf0}6 zxZla|rH&};F^d@e0KjW^C@E{QwJmJ`h)ZB_RDi7rX7>9<-DM0d}P|`uJ(hVXNqo zFI5Y4PGrDj+)0Z(w#I>5s1C(Hud&T1M#A!zpl8 zr+WXmGA&!H$-1DE zxs%B&&#=^;CaBAm8*ENvJ!vB;Y}5Bu%EuC#wv3Y+y&N9$$L{X8!8M!nv&JtIE0ZQo zqU!C9v`OX#4QY}EZbWK?7$%dYeup^5>gY3$>gc)cRq@6@?uc%#ov4v7P%Gj3T;bV{ z(M5)_d#$j>&d!dpGAw;t*D|ESSrI;O4;9dCJioiWp6!|M<*->u^1S%-hzJa98#8Gp zgHC_db=%|I9)zY9r>SVm?l0Mi;qqI5-%pRgK+rpP!zD`+e8Een^RvuflUh_6)ye@>1FJHpH~4&3bUG;f)O}hzLUG z*levKSLxK^dQ#)Eca}H5br#_b_3wT+Hau-_dVsG)R}qicNd@~TXuo+sav&z12Y)}D z91|(Zs=&&%12vzGPY(|MTxj9vudw*+rSsh7cy5K^RKteBoPvBRzQ%^T9o71KnC$poX5w*h(t1>sCN$gC;5|xK#T;$b z=2+~mQxMu0RYVj5&axh2CzzaVS+%(04OiOaEi<(NSEwQi@z*SqEhqb+LkZ32-_$S) zhHjGOb@*2h7hMyr6Rw_Tja!(C^_{VC(WEfht&36cC!sX5`0eY}`3&8h3p~B9*w<-v zs@$ANW%Al1b#)$NG9Rt*cPQJRG39QqUDc~szSl`iNazc7J4qE&@WmkGSD$KW@bxEm zi3@g|;u_#GF&7ARri~kX@x$|akDJlmo1wVS(9n=G_8b_4msa!j6WHyu^?Df=dSMU_ z%<%4xj^Rd^V-7uZM`3{KEayEMqhW4Dutztk(=h18 z>a|>@4Bnn9u~E{ADmUOah`Ku4D{xO?HWYmf$&&~pdw&Or@LA>ZJ9nC&XzAliJR+cZ z>@d>1p&X`^L(o<=yvtLOXKT0ud*|Z+T>=fV<%`Dj6A(RyMKJ|38(YryB7bA4{ECoN z$2@eW6h|IXluhBgZpK_Eyl+?sU7R%~x)zr#ci78bA%H*qw_Oh25k7dv0^`NzE;(vf zD#}c>d2wY?ZKYNtAbEmC?vMm{-4e4Y9!WIow2nPV&<4Ua3C?Th@s9hpdQ?lu zVp4QfVT(fhsD_b3s4y{|V>FVUO5*^Vxiz!7JJGQLU)#uSsxk(yNgxsPn z4{&->>uZBQ8~kJq3Le=Jfg8x^J;Jl}?o@mUY%yH62oIEh}5^_%iU5 z*P@|=3-TIlM#}%v-u0_c4)XPx88q#LxI4vmyG9^nTk6g8S11%`=cTwc&yZ_9BI3qh zN`zM8TL9VCOAQ7(x-`kVfxusH;)O7);z3i zJbChj%8xvS>QBR048*6O-efj5^6wEon`A4dt=;Q(a6q>?W$>B`a6;l)<>5MhMqNnX z%dV`cnJ*AV$mE=vUtCP{#Qudn2Ihqhy0ByUDEOxu@%=`DN z=k+5AB&QqD2QSy}+;G{%1jBZL;Njwi^CK2rlFn3Vaf_~COSn%REj~ZmUE3f}6TPwQ zB{;okiV6>xG8i5MYNHZ^82#o~bMbthtKkxx(P}66TylQ>x7#k$?ITuBr%s2rpfq+oLxghN-Jh-Ctx*1{yS5n zu#RewqJI}0sgV<-g|{Vk2}43O)mk^Mu_`Wm;`3WJ|C-af;>HT|y^~Q!LvM$jfZ~P^ z^AMc-H@XTP#uM9i6(H8WHcT7)k8?2}e)WI9ss$VM{#y&+uaB145%`=R4R|!5YkyF+f9yluf+cI* zc@%t4M2s-k+Hmag@$nK4Y6E2Q71+4shGlPiEsI~jcuM23%P9(X%)*pPx`#5(&-VL{ zHg`vjUw(WTOLz(*iwqTj&Mt2kxbajbW57>-ieDxZI}Sa6m8x9x1?S=7eZ9RqtktLk zkeM4I!XIu14(V+em`mcB&yIwJO73-2riKA;gg()d`z{pe|GJ0&{t@dZ5)Gb;;>h6n z;qLtHmj8~P1A}d;=DoU0Md&^>!_6z*Z1>qu&IZS`*bdv#JPW;Z$V*+seUCoow;wiX zQsYH9E-lg}Hx9!Xt)Pa;({q>AX=Jaz)xyDJ!TBsv3|Bzp2T7~UIkBfm|7*P~AAjfe z#@BTgsoAtBX^OFKJQPoEtVZ6y3o5Pdmhk%DEB$}2QXNUoqW>79B7yzBaWub5WkHAu z&3c(UW}Wd>{g!U^lcfW=(1_j478BZ$q^x(${E!BIiM>Xhv&)TlPW~0IqNp1SRQLPj@cs-IH{m$ubcJ9l4a_5qP&w%aM zFOXm%@{pNlQgL6-7hW!_&q+R9Bi1g`101-9O|6ZF{g<;5HhmT+=)t?TzCQDsmzw(9 zJ;!=Wh{q2~4;B=^hvoSD%V+5YMZ-F|XWXVUJcIBwEd}*y$27T`I&-8c62lm|%hwBh z+H}N6awIPP>TN^D7{u7d$Hk{tr#H4|@0NA%1-VRqWO%!jEDC>S3Hxqntw==-x4MeO zY1L=Za|FGdh34CkQ}T4U-DS(Zb34Pj4yK`}TXSnWPm_ACn#4`{{*z%%(=9>sP`@^; z$$xXdLB0?I)fI#Zrk&V~t#=4SK%e||V_&g+c1l`OKx<`}p3cp@aWs{3D_}}o>@ak{ z50w7vzI;B?usF`WnLld2+GgnM`H1+DoQeSP0*>vUZ~QSEf`Y<3!Y!W)T7|B_U;H1U z*6GWocG>QOQ}QEPBLl1(85#Q49>?Hx3bAuRd2EsT6H1>x{Go#%!k<5L1UPwzzm(W^ zy;!`z>*o4qsr>z4n;Yu{Nwrv8DhQvkTw~+*d{G`>Z(!k`Z2F9pYAbW+|Z@iOigky3MQN$(?RWPMp%=}2l39yaq&n&CnXc2|(O(ZYsb1crp9 zF(o$)uf^)^k8!pWwC2>Z68uxl@jq>)aOrcbbEsig@CZ6W;Zy=uiuI5GI<>ySnwmuO zfMe!RKY74QO8Vu?|NOmw{J$_3_74_#YHI3A`wh97YAfQkwY8DaEa@+wvef^zA-!9$ zy&-U3P3&>_|F2gj{QP@JNLIFNBFYI7i_5YA`LB0AV0*KA zW21fj=X3mjThhPIH&!juM|7;(^QY{O{>_*E!`XqSh9^LzEfaqH-zMT8zxsc_=_3u- z_5a_{|Nn%5_5S~4%lU}>tglZ93=GUoPp4#|3D&s!Dx4`V-bV4C4@Z?87^T8^viUM> zxO#7v6ioT@mNwlDG9;Z8(Kp4Sef>HURNBI6Lx6$_Ef9Br;ajuCn{EsxSGpXV%+xzD zh`oFF0hCkoK>Z*pI(qH=81v)_$};=&%>mZ?`WGyaeG=D;G=HfD9&tK4o_Ie@V(~91 z7;O2=_Lra$-)yPXr*eMV#s{#6MnC~LS1CtMFclN!gc*Sa<)Gx+XZjxyg#Tlpy!(;J zTsL1l%uF7;Do`!*W%X8i-=-tt7zETxVq=(e%qxaLGwUd5j(k5oJ{3eTN2HFo-f@eRzJc zzGGtw3clNK8MbbO?tXgSUMYauby22+6CDcQq#qnxWRu(Z=?*73Vs&3N6Q7WGM~E$A*KshuW@bvVEYKWnP_X`(EE5J zB7j8L;e)xBQHgsEL24rhjj?g%7ZD z3kyG+vFhvV$;F1U+}>{(K-Mu_Ii%N)l4esCNN7ty?Bp#vPcWU{FdzDyDT5iInG*X#=$*d zHIQS7DWQn$H5$6wb2(UQ<8VDOoxcG=MXo{yEf<%n8j6T1mN%=g=^UgG`#)UA*T*u6 zTp<9E#(&GurT8VF6_k3dYLgBQK7nnFAD|Ze4Gn+PY;RU;Th%TOF5@c;p|}wR@GvaT z-y9FHn*kdCdZD8=6c%^7Gd0^o^^(l#hHhUi+J5(USw{9|zFJ9&DwD%YO3K#`iQLdK?O8oA zQT&(gSN?|wjD!n+X!+3wodPZCjV)y`yzjPLLrvVL^ZX31fB*dd`@}i>w*u4=gU@rf zth>kZF-0^OfG5e195=SqcAvhxYiU1e7aA6p16trUkNy>uf28_!ce%k3TlKLGf%ivZ z!Rs$97f)K*g%1`D3$gzFa{m=LA^5g7dt9S$OjlW`wX@NIX?kg(9x)2Ej3lqZKOh*8 zP!v}@$Hpe97JKp6o$>n6$R>O^ULQ2OKHIC$x;Z+A9z6Y-flY)!fcUk`ql;eTKg46; zqgL3H-T8*8Pobdv?`OP@aESM*mpV4C;>|;NBrN%?c00dd^K?pYs>r1;ijDj;iJCN&L>BCuL{0E-(3`FCJIfNhR+Vm%Ol$pEdQ4~5+-(aE^@cqt5G zu1tUqCu&byS&zMd7!CQy$LG~2fU!|@x_^8<&vvIP?bhDQUXFr>!TKd%G+G9R{^kPB zULfld#bx>XXI&S*s({;BW^qnxagtgQx^ww+o6@4e%et9ZhI6GW>@Eh?!r_#u%uK57 z(JVBF&Ed8TNRO#?vw@}>G}wH8@@DOrHdR68!M^}VPR30F2M<@Veb;Xs1jXC)4crss zAKoo^+1Y@uXA%FWq@zz=FaZZOj)>Ddvx#|iX5O}NN_7?%Sf5leM_z|-Fs@?CO`~Ith-$#~;X0igd~EXscex1m+-XAz{}jrzLDH`8 z1yIlW-SU3)eg&*dqzgbLtLuU+lpbAqb9Ra5!;?e+OA{U#>tb1biUV=YfDh#8En@cFZR~RJ^bf zo*}+Htz`3gkaz3$@2i^_%eGAH>b9 zXKKwd#smj(-%Cr&?$S(;mm5p_d-T5nMzc8}`fNR04rIei7%d2%ak|7b7k~5&^!dlK`sO1iL+V1H42`()z8rxcw zV-f*@oSdA*6xVIK_AH);)X{@(axac#+b%-C8+8rKtNjKTdiN*Q53@2MT?GY@E>l5D zwL@}NyRWj58WkpH(DD&>3fPZ_B4c(41EGt*2Ozj-1NTa}ptZyL^5siekioD@@u@|a zTXA}*jh`kN&p+_s142Hc$DeMEb#nk+BI6o~ybC}I<)rIMn`@4qnqL*phE%M4R zdl#k$9R<_(4w@z>O-sESnR58PZ5H3W)h#Ih4Q~4Zo8{gc%@FRssR;xj^G!ai-Z1rR zJE{!w8y!v*tS@-S4Wn1?J4H3TI^wcX{UBuFJRCA#`#pJF+KiV0w@|k8S+`Xf3=N1!4)p6S_T$WU6)qYBoin^OTu5Mn}>6W{8%JL;}Kiv0fw=L|(u}<~8 z$}_?F$XL_tI6=@#Y$=vzB7kqB1DkJ93z!_a9e_f)F@O--+NDdn=`o7Z#M_Lpy^#L*?89;4n>K|BbGL*B^6AvNVv(vgv?3`E~gguWJr65Ao_PktsKdGi|`>jA- zf%*I6ug5`8|AnvodxPr!b_sa1^n#LRO*F=(A@9Q6skur%^W#o@yp%s>b-$TVcNZh%JLMc8yKw2@i$!f{xU$z3fEm^jYhM z!RT#2hsMrHHP%86LhD*C6%i)?=r^#&V^h#mo^Oy990I?2cMVgE$@{usU@oMpBfx%k z{OMI{Y^=m?)fe@xeLG;;u7iu~)`{pD>~DD7Y+-2S*eT8?97ssRjBTG^yGBW4~po~dt0=fW61 zZ~hkHX#f(3^lMls+#qX2I=}7Q{=c*T|G93hHjhaJ-D|}U1Eu@lV@#F1o|@PH@gqj> z*P;92+y@mnPhwE9+AiI(CIZC~Tzvf5o;~(I`kePtp5wU+m5zIw*Tu=}E{~e^Nf+SE z)M=0xjCQQi9sQpdWxZu3d2?M@QR%$7?|2C$2eI8=e3}_Zl-YY!5(Me;e7e6XrZM!# z*|akdcb{Wy+pZQR;*|Hj(CtNJhCyuq?=OL(E95E?Da{Pqc0=pX_N&t!6=^%#6)fh@ zTUoIYbNEhUfT}FFtI%2+56&9;Ba9!u)mAg~kiI`Sn?S6wkL~f7SmD9@qsO{j>V*F> zx)5drlztCh(pA!_{1ZUq8v$W^Zf@xh;Ak@T>W$SFKF^rtXykWyb!dV}&nTxv!S(Lj zhE&v_a?yl#Vyx~)n$Y9o!aGoxu{i#KNA%cOwwRV_WqBFh^^@%D_dIHj{q>WHJ5`Mj z+BVf4`!hPJ!I`o5V6f>0PxK8iRS_-^51>)|=2 z<79TOJh4#=Ql~mdDkq2C0hiJa&qOD95}sDt5C~*U-Gn|>jWzcY>^6>XO(WLF`T6vN zOB*p-)(TIVQ^TSe)Xa^s1BFxK!L_UJ%r>`WgbBU}Y87b!@||p!>~AnY%Ct14{=BC1 zN8a9g_$Pc5s4{5&_7)`>(kFslkstp;Q)OUl#>qgd2>-TwmC9{H_E2N@gXNwtxXH7R zemLC(A;E;Y4~@lWX274Q&SCMl%M{sI9Q6tZ8>C~I{wW$Lj4J!@V8GDDuV^;DWcs#h ztoPulo3Yl`7u}M;(2f)Ub*&S*9)Ci)=t(otwu`)Z$ z@d`{cKEs6ff*x0KzeyX5AE{C$W)9rMR|xT4d9Gt&Wc!o3_MCulWc!K}TiazVy07fP zdyjXlT3WF|)E05?*fv00bz&%)K)x0`@gv$x^~G3FGPOu0zU~6PmCG_QBEmp#KH|LV zJRr-y04)QCf5v5~(%-^#d~}2!U#=hNC5u;a$P0ge)YNW+{BJ_G|3dnFmZ&Hi-l~|b z55&`#NwXai{j_r6kllRmi(DaxpG&5&81yytad0>ZTNQvwW>q^>Vm zQ2qZ%XFD7YYzj{ZQ>2I(VSxF2^}4G%N7Qa;tnZhpfAv&(0Fn+T?DUO_fs0v+R1zF{*>0V~AJmW2TUW5JKP1;A`3o-%& zQB|I~tX#k2`2dI7nHZqJ>v&VE@N8A&JxDJ_Tn)=!|8p?=ua!CAMi6j4`5?Cy4@nuB z^A2*eae59LQsxRTQ-05I-Is2plm$#qe#cYebbuiE5tB?FRt`&+(cR8J&{G=(_A19D zwRam5Zvu)M>`Tl%&a#CUz3!Vv9UALkfa2Wep2xSbzP@clc|bi@?`#qkbS6wdPA)$d z?`3IOsK0|!_z~~?PCTHAiV(MsaYJXXkL$nE9^-fr-Yw#;+gjKU9l1QX4loQpuAzpf z-=iLQ0V|`)_~{CWMVelr@#NI~&4y4>1=kdiB`)nvWM8-tWIZR{ggnX7k2<=H7ei$P zSkvR8>yU|mEHxsrz{^J&iT6jxbxU?3{npE|O(UkRng}BnO{af+!RhnRU^Qd|TrpLH z1T`05!zOnjM2~_sMgRqTbF?gqyMd#XY9+dI zZsrsD(fXu^=|A7!{Zi%Rw(0+>CNA#RuD{LwL5Od(N785OudVV1XJK9TiIV}P8{?dJ&cQ^3QJ2J$O*Y0>ql-BH=At44!W z^xgf`HiXFE|M;FTGifIthP4CkNQUcMCP}e)+dtp%Bj#Rd-0#mf=$Ni)M4OFeE0~+p z+me{pZ=|^OC?TT%q2G!1T;;j#^Ir#tIJn zX1m4xmU~ap*i~!CplyneFDUkRN$H}DV)uZGuB2n7U9k1C`dLi1_(M{tfM>hwVx?9_ z+d)D~?s<_`ej78m530KjLnGEQYZTUxLw?}_fo2uAsoIg~ewEvAMg7muY&!>nW!j$s z+E}!4vymn~G}vxL7pK~CVzl%eSU|#Cj`@G}K4cqyq;O4jf-u2hvnw^$yaTHZjHX%B=q?C@|1;_@_w1+qUVUH%{H*S5swL3zuFuX`IpOm2JqKI6JCSrB-PjHr@k3a>PdErYbTfrA*lgTWoq4^yEI0B*3WSQ0?ehuY zp+fPT#P~)3j3KqYL;!td<5dsBd%1DA#rv5ck?S>h{L%VVAi0A2wNiwRhY6?xH+Fo| zU+xH4FO-H%=RJfzTiZi%tfH?9Fns8_PC=K+{Qecl->V4_;)t2`s*|1)bQo|=9j?Ca z!akFli!K3L&$_!TEyn$LuKRTxpB;j=MR(}yU;LQY0pD)Vewg`@tAnLqo2MZ7ZMIze z_Issqc=(#f=mad@tH)qK>XKpukbqZ=WzyLMG1|mLEDPMidxNh_^1VH44KL-R+w48)tBgcq4zJ4yn zDBEjO|4g1QKPo0B8(S2w=_SG$X2bv-p+abx#b+rR9r zF;nvy}!UP{#NE`nzxHv$H%1U%|TmpoO?39>MFn~sErnLQYvGDh4jPDUx=?}ye=?;xaSbt&F zToz|`leq;eGsziGxfvPlNwh?fM_!+7WhBmk`Oyx#RG*{eP(fOcZ#>TqHRj+_7lfNMSw{}HRT^Vu}SYV|B0pisjSrzBt%wiS?&U0W!bt- zuG1jHH;^fHg6X=Inb0mXLZ8*QQRK2CESrf&1Eg2QprnV5ot^UwHLGR#V zd@6k<)k1XDdh}K@ zr4UXeG|}1P^hx}J`<(+^V4JQ{?>!g|{gB{m9!=seu+^85wQurbZCF;~+4Ngnc`WZ; zUgqDwedJd|cVk1BIZO$v_Gu0+Wt#sGM?OYH5cNY(BNK4loMdy@R+bIVWGKZBHbJiH zvswp&fg(qU=|VKD%5g0EzINvK=U(?YQHkHY?r*t6)M(Q?9riuW=OM-G(Z{q4@x_5_ z*C6OM+ixSqFt1ed_{H~}mXh+9!WdCBkf;slxcDWwH(lYsN@yF^@&6F^)=^z@CqoT>1KXf>!SDM zWn_2*xh6xtga%;4Kf!U0A@HsAsj=nz&F^H^G$a*^?_D=nJ^ltzx7Ll!YlN(Q^GE{h z^;^US2c6QQh`g(OkiPk_84;{A;SR){pmmObmGHGSmQ6brc=AeG+&Fdytl;cT%130f<`cV<@E{9+fz0toMK_-|^>ni+%gf9B7+cTP9lX zNFV*16@)U6iaMIfEmDUEC3;_+_7WB1;?rOohqzDW1ilxSX0%8?@x#pdz->})x` zmtr?kIig;+b-@9Yni4#}7(&sTs>%v+!I$Jvt!Q5EXWHP4v6jkZb`{uDy^G=xIj>>p}uHbh(j*~MYE zy65bvu6P3_PfNps273XtKz8b0RY1n; z2Z-W@i{69MFOZ_qcn|JP3s_tOU*x%O)czn=t#{;ra{J-3 zYeU9Z7ohql>VBT1SBa@})NDw#80?W1`6Q69sHC(P>~+6KfeMaJ0$Ek&6UVoBSKJY= zsx(I5y&-(@cKE1Uz11R_mDN^Myg2|D3xCcY738}QIP|*_d4VddBmB2p`S~|AinH&& zUJ3Xxe4pVfT_5F0zYS^D7| z{}H%lON4ov5WCnP;KDz)%9xrRv)pNwfBJm!2w>#qt&^$UFz^1~7bU;&Dj-cEl*5|K zhsJmIHps7YcvVfB+`VrJF}NdY0OYvSN>A5~2sG^?p}3jJD?m?j7f^`TMPXLjct^=E z^%b<|`Kk|Lxj;wW#VtHB@@{uC<=l_G0G!RN?cu55pXw<^xy8W0k_Y-rv@RFj09YZt z9`=(X$nE%347plJ%>kg8!(Im0+)_toHW~K=+?!oAWpZ}5L-Wl71ez(H&v!Kn zqBVPDDvD15cKif01?N%f)mE6T-0CIJC8;oDy1WNG?2G}#Qys6tD1!G4;_7(m^GzAc zVOJTU;_N)#Lb00m`_d4jyt*|ypp6F%RS|h5_+z|%Ynllz#KbV#bf=~gM&++|O~pgx z18~617^ct9owBI=6JJ{cC@mbL@rFGC5e zMzGxEUJS(F-hzIvF8HO+`Nt%P5P21%s;X+&Q2Fb^$z!Z$zD%V+Z_2>HfS99badf$3 za&gbK5UcB?v8z!BC&#yj!&MJ}KTN9`>;y5VW{{o}UhV`DWaJI_-lYPb=7Pz4ePv_^ zse912b|wI?#bRGf#Brb3w3by9k^x2tn_k>TE^Cg$C4PN_Uc;=U?*d& zc^q~CP2e8c6G0&JaC1_>_Y6c)3Z??ae88xKTC-iE0CbK&bd$#i0F|*>)q%+^y5G?! zb4okIH7T3V2WUPCxE6$3=iC84?X{iX%2Je3w%X=E_;^HtGP?ZQlI`X3f{KD7V80p@ z!6uCBq~7K7@WUwe1KJ5@7D$y$1n)K5CrW_p{lN$^=<_CkWh$G|_$#5p3BebhBW{iD zx2a6jAC`8_vwaHnd_IftDrhWLn0vi4cR~5Ky#SfSFXzF5&|H4 zLO~z!wA^{MXXF zDT?N3$br-#G0th>qg3x)=!f@;MvRq7_5@-?bDdGK@0V z`mua$cX+g^Qz)eZmy##@yFpAtAju^OO4fbIncw@ODH;kRFE9TZa7Rpk@m;Lz@Mbu0 z*-njc&LL31-8G75;nu(WRNE##t|u#m2NgOVqHCT!Q*H#v4zZaG21dE;g?e=ofuKLa zSQc_bbX8Vi-iW>N(=93-@`qj9c0-ZpdVz3m0Y_;MP({Y%@f&&9(m7G42U&Jyp zihNqOsl~V^3qmM=WD2T}0)+2CK@ql?mgerH>ZV_L%;(TnGAQU%dgbb2b3~Kpa5Xd; zo9PT^bA7BBM${v9UhgZ1n(!i`Ya2QrsKG-5l+3Bw)LJxey4c+idiuj#o;>6y?q)XK zTF5O7mjScZTIaP&ZPEI&{YZ}Mt4JjJS8T8BA!zCkJTg8pR4hgv5KRCEu`JbGuoww0 zsqyh52Q%gQmP?unsDUWcfQ3ly zFSL=^zH2+z^FoRvVfwADrz;@$=P02W$OM|=V}Vg5xYKsZlu&O}sycpY$24nq;rMbA zX}^kwP|-A-1DvCQ?L}eEWwG%a8+0D%UX+H{>N?H!dU^g5HF1IV_(Y={lS0NcYj>?x zcbpAe8KnNXdXb7MRLDDr7Rt+yJcWCUX6G$hF?H{)VD0o{7w@>RuRvWuP#KB-KicBR z$g=>eAjm7pW{(YMi2CYulfU9w`{SUxut*Mu2NSL3^m-$B)^`41Vy0 z?O|~x1zp`t?V$&tE9+!3a_$Z=sfHQ0ard(oSS&H;Wc=wZmR^NuT32;|jngc?k zk+sC44xs-uk?OQ;qG$NS67M{v!p#${H|HZzyN`}*O!sg)TkKcwU`zmf4qAsphSx6F z1Q4ICCVFd z+iSZof`kD_rcIHWpOT&H+d8jccWTXM)cxaYE6*aNlKuQ3Voq&YTw+RxbmyuJ$*=YF zKZxoMXMUDEl5=wJ>w#Rcpw@iQu+KoYfIzPZQs@$lGQD z8ud&OfL@v7k2p-~MF3IZf!syD0#RXpK8%@<^UK~ z3b#>WCLV!F@_g|fnYMd+AMRl~LOHa`t{%6X7auA z&j0T}KHe0Jrk9lO9@bU!4^%7lAX7c>_oNT*+#r=*k!c#m^mIAJ*Wal-`rS*QI@Lew z*LCjojp*Chsh4YU2VA);u>o~o<93o+3L*vl4`?L(!dl=U1!d*8aR~{WxYCP7 zqu)RD>f)g{N_)p`%vaN7Vs&kN<zz=aum4hPE=3%>!McJlC3>Q$JFR-fUS$nczwv#$Wb$|h8 z4Sjoew`Ut~Kh6aO1xa`6OkR3cRV9X>B`U?mk#atFFZk#8Crb|76%88oa6wi^y&0Lzh(koP&nwmBoCJp*bCK-r{Fi3wGsyAA+lasWaC$pBrD)yXG! zlTK+f)e3+(Y&ww_)~Q9x)ebNXZ`$wUJ-#gH`7V0Lg(C9-}F96923kH|>1Zq6frkPN3 zUVz2b_8ubAWdN8mAQHiv^uE?&A%2$Shpvkz#;YwAVnA9u6#R3%Wz!LBK(Hqw*=oh& z%Czs@KQEjDSnF8`Lmy|juSDLcQerT=@W`n3*dc$o(tUmr5z?Qri`3ix1SmXEHne7j zY8i0d?LYfKvVbJVe#PVFZfjg54EGZ*nmYSOHKblzmH4UL3V?)U4^TxY^9h?y0Dz;6 zwpb$5*`M*;kEm=IKnD67Q}BEK`Q{49z}8X3}cud{-HI3{2pS2 zZ3Dg$csUl+G{G`nix!q2`rjFJ9r!rBM*ls)52ya^-O?4p$3;GXrC9TQ2AGzl7jsiD z>FUbS@^rnxI`K-WE98H=XMOKA_-e)ta5ukvR|Yz&p9EkCAsKuCQr`qXIHv1$=7rZs zX}+v{|9jnh_()Fy{C*sm2uiz>dd_VcD@h(BCzsmS_P8&x^&=u2iysIG05L|;_w@lb zc=Y4`^Z>RHz#<5)6Keua=^BCnWAlwEl*N2aCV*0_P8(a{>QJ3lzcD)jS+C+jUVqh( z#&X#2W<I0GD;zQB>-tUUE{#=VN6MQHg$-_bnEXdmr&W1%vMWsm{t?Y|D||A-u+#m0dl12__dR!VY85{UAgQ}izmFm z3@K7w6fmN{>o0@er$A8x1V1V>ar240wm-^LTF4A8hQZOOrDgkFO6CqyX_^18PVJoRh;^=toI9ra&7BH|-_4>27`_Qw`H5K3oJ z7~>M;m0XyVRALnza&a*zA^;6ZVm5ialG?zJMO)1tLnVXvN+>T8l2f*tQhC%o%-HG0 z#SMpDbBe7oTsdgygcHG%^ zI>f#AEOQTM)vs+`EtZz&1YlWW-}%wjrh+WuH@LOo>iLK>*T zJ*Ozi?K45{@g!-)qy(MN+M+h81-Aw~n<-hTBzH-rC@DyO@BV8A30I9EIwVsb&7~6& z|ID@XgZJ65A}*rr%dh%MqUiVt6L-;&#h8xq*jtdbTi~ckH1jQh-HS8_{qFf3Igp=U zQIA8IB1g_Zk6Jtbj0NWnX^!b#VwYsFv~+f2jy=yfWeNh^-;%=ref=3_u@$ypMuiR&%6>pxcTLWQ` zNLq6OD`1Xr`|n^J=NN3S=VWZ*W=3?A-n^jcHN>;36ICj_C>W9+IY$J=Mt!nXp9@+n z1pK86NH9}KYrL;l`jZyFnJY2bC@)kKV`_L-6)7uZAzJ?eEp1MsF_VMYEUD!tE06v{ zn_!2bGCuOyhGxMw`lM*13T~oJ71k0<3L&6qOWsZTF=IE@6+Kx#Jq+@_G_E-#a&q*i z%EBwk;6YMSu)o69|A-`j-Rk@r-JAI_JpQf*<9T_Y?5@w>}WuK^I(} zYK@#28oq$O59(=nD<4aFPC`s6j(+DJzUfYCb|w6s0K4BvnM+h8Me(*;@4GMs#*6HG z&Q5JQBFv&$KE4q>%64pamGhS3)eve{QZ}1mC@4J*!t(*N_e?qkVd%pm;FWt}RP@y- z-Fb!zlIq~B=#km2Ul@`5d%%B7jQ@E&S7CwHgR=mF#s9w7e=t_RCi5Ax$Kfu^kcnQ) zon^KtM%O0?-%<82WKumAH!Mz&avlo@5p=>m!tjV*u&c*DWz8Cqc`1^fP-Euus)Vk- zCw(Ec?qF*d+A?RBrGf%;r#I678a#mJ51}WSfrnu(Bnv+1!{|w_H~voEz2ytKb{iEA z0(0h%P!V*79&l_mbo`jrbZW4x{P&SHyO%@ zh{MciUpKb4n7HSKxzrb;h`b!CYnL}(iUttf#?UnPcevf+9&;@O!@d8-ahV8W1lkC|Lh6NIfU)15HKE6STfxuOkS``c_BKXLhK^ePp_9 zO{pC`!x-#e0fJxXm3U?Up%VSyzy9A5ncD}>aZB6T8WSTZ>!sb1dcOW!@U>Oed561) zAvEe-!X4z*|9An65#ZnzQr+kqh_*Qj$uiTt=+~q^>7x7Jo*=?x^XH9}ghuj@D!Hhhj| z;zOqVHVC$qR88!IwzDFH9dAB}Cj3{_bKnP-(}5pP|BollS(0z@0rPl81k$Zuer;Jw zvBHcS<9LB@DsGGZ3cLUFR4XsfA$8PfhAq7qfNV7@e`+vRYzYYroiBA>D!`^LKn7#XrI!E1; z)$-ZPN2NG+r!#`#fu|pdl**5y@?0n9A8DRDT@pS3!OxonOb5~!wxg*MRQmSf+elg}GIeGVQSSo@+ol9Yw}@1JhC2<2zWk_tg7sZWE?2sG8_|MaQi}_+ezX&XpH`q9KMBvJ z6N-CD-4T3>0#3j&GAdJ)aYl5NI5mB77WFfteJpX7K2_x>QSp8h zaG&eg@fx~r16IdL2*dx{Y+As&fiQFk&;09q9hr=*g23?OCDFJN%J9Ij%-9mz;7^|_ z%l(_vQ2^^g_*WxiQgKhacU;bl9%tQ{fDrJtphduCi>$L+v5fj?IvEJG>iAPR9IE=P zLBWVbGa7@J9;|Y*n-^Yor!nl!5407vth5xm$?ai{W7pXm-kiu>QK!IFg%Tc zPps#!)ExGm6n&DIp|0Kg6l(h~{;wB>^ zV8fj9ibc;Mt4<{BJ21n58~NwDHkaNP>VcIi zzhFBVmtI-A^R95Oi?SC=$D0Ve7i#H9A3j3l!0^zF-7Xk$2`-j}DmW&%Ro@mE^x#Yo zZIyI*Cc-sGxa*RM@TE4-JeFW-3qF(5QJqmxbKGB9Z-7JUQuIONzd%=KNg%G_XA27c z=iGXc$STQ;+iAZObxWYnjD93h9Oe}ehl>c#Nz5@AFrM%qX?o!ow;Eeq$qRo4A8x69 z0P?RZu;1dUn*g3d+&RErngC4F9Si}26eu@Pa=l7)kpm&9!|`%DlU3xO^T_NCp3)T^ zFTK~^6T2*U379jJb@z|FP>LHMfuO9d`CrKaSpxDcgx=Pi%Cs;fGK><~&uhzqeh>7S=OX(B zi~7Q@BwCo}89lWP!Ep%U5t^>mZ0z{5?l%3VVLoxFhxtgbfJ4tziU z{FpC)e9|#BaalGKw%vTrO90rhz#wiC$q>#iBd|1Nz7m_O8nD2+y1S_chdKZ^fE9NW zJFN;GP`tk3+!81R^UXeQzEtD_+Kr``!M$INkoLPL4T`P?>9+62P)dyTAyexv(h)%P zvMMgLVc`S-lvAAy$kIVgf|%xs@Kh9)8&D$#$DpvMfKU7PiUsg&FmH{txM`9nlFE=tHFHAVPba(GJK0) zGMZW1#ww*k;+kGuV(6M4BDR+rnlI8Lek88m?h!Wsi0%lFm%FXBg)1(|D~dIKpDUu^ zpvR1Y!v!G^mse}NsP-MlvpJ;$=|WCcIxo?>NP62%fgxN{Zq3KuEbyx#o#&76fQj(;W z*vXLKY}fwv()^?r$ch>7bz6=B(NA-QUf{SZmBB4gCB$d%kdwy}_tQO5rcTC_qm!Mk zXDHw<{?+pFQGa!UP!|je|4-GoAKaew@+8A-3+qc-nMB^0*7vusKFi6hn3ea|| z*rs(OYVMlPGgesNi<0GlxFHsB?XhHBp5o85?2sG!kA_WJQda1m=TjwQW*3s{R4=y`K>@i6Nn-OtldzTT#g_pB(>TGK)ns;ZN(8e?C z-IJF?9~=zRU`+3u--W5+JSE-s{z84GRa91#HIRhwHacav)ovm_sVQw!>DeFIqH{7nXfoTlILvY22Ij&HmD0 z!9SH#fwewt?Z&UEf5v?=HND7_unm)1Lfm*Jn#|t*MC-^yIXYct3>FqWAx%2y>Odjp zc!?@#B{PjwC5>BUBgFq}&$V%Pe zB*-lnHP~BkF#yT2(2TGF8x9Dmkb_X%F8vOZ?WfW2!$ zwDt>Dq_(1n=$pZU6mCK>YKEcwmW#IW@W4biM6zEI2;jH7&h+1eQ(=TNl)7I2b9!Ds z5PAb}7@@ZnejeiMP4ZJLBQ);WtNDqQ?qyYznyc&uWubX{D23I2L@+}325`|}e9r9U zd1BWE8+JV;dY;=++-Kc%Y&3m@kczA~L3_HnH{1IaE|pJxs${Ro9JcI~*x|C1FIPlQ zlMjiulAU|fB6#=Uh@~{KDb+ZN}6X*QtYqiVnCIvLoQty zxI|qRSZ70lVLbrnbbVU^+g z-qM+uFqa~V6%EJmnmPm;)COS}mXQ_NOz{%QuSQ9L4uIezvRkPrnpj&z2J8Mp!uPBd&BP%JY{!FiRg#C>o=mWt#a~2wcTt+9qo<$6<)+>7zuBw%qpk zH8$tnwMr?p7oMbd=TdDyme=A7G#$GyAva$8e&T4TCA6CO;`$IO4KS>(=vJO2>0EE> zHit38dEE3i869SqYMv>s!p-amY~x3VF*YdVZiob>1LwfIhVs*MAg$VLiBDEv#;>bp zR&vt-ZcD~=5V6-P$%iPXxr%Oy$&0IXH-4zg>!!>kZuueCW3~H>(uAo4J3@=GVqVRw z5x3a+njl|wieOxBw{^&_)SG93oKoO&u|2@l_HfME2yg=945`;JR5bX3PML3+*d{7H zpdjWVlC99EJJM#2L9ALAcdVPIu(?a`!3U8?)2Cu0zq|(N8d{N&J2(s#SL@fXHAGZ8 zEAx`jt6yWfxR#rY&wd+qm$e!FMR7{eLdT?RBgBrel&m=;WxZ{&Lk{QA%O5WL{KLM2 zQnf*mc+Ao)#XvU}p}QB8uqCH-T{iyBMY&FgJ1x&12I`|(8H_JnRp zUwA(RlWWduFZh^E6l#7s{$a!XFgJ#e4JM{yxRra^>DxKG%3rQX%YCdGW%Ut5v#wcS zxLack)BTmhu>MWwXeMS@E!i%B23q|*)ql~vp{|vZiYkZhuxMdvvvg97x|g5y9bEP_ir zomt98H#LpSXKFp`SH*+SpTh_jIy7;Uc`yn6RYzT7QrpE<(7VE#x*-cYEB9%`mCN%?oL2THYim6?4<5_fo^#k4nsoq=d#gJaf9dU@ z)dkLg=&q4EX}L<*nV0FUkunn8uD)vwf(LfX6z&k}BJ&w4hy7YMsW&J(Wh=U?O4|}k zcPHg-%~#E?#oD4~XG&`QYg|dQ4;q@Ph$=lV*TfBn?Vun{oC!WSSj0=elQBZVk=~`%vPh1M)!26kZcW#Fkhb!v+M!b3~|pK=YQF` z{$RdI!*h+MdbQ(VcV{8KdhdTOT;moU3%G73$TXY zn7S=fQIoIsJ1j zRHZI&<{xHI2Q-^Hbf3PH5acd=c|M#DJG*RYbHB6QVguwY8P1;2O)=`VriQ8>0KN%0 z8yc#Hf0ZTE$s#$x9BUO`naEVCE|?$LrK2WJ8IRGUB4J8G)lcL>L2)iZWC1o1vn*XNPRT<}dpB>!;-TsOfvj0iFkf*r{3uInw39CdK!wO{uh=wiR#0<;yK93>`nQz{Q35csZ&1)Tm(;peu#dP<-^9!i_rH_7- zi;owrT6{pWlR3r6GOQW|Dxq!wo`}!*IM_F^+}LgVkFaBoYivWNd%}kl;#GeCPTxYKT06k1SSyh=8wfMAmQUaj zw0L)cr&gFnSGdumcc^Bxwk1K|bM{Dp>oOjEiLpC_^awlCdS4{b=t1JrA*6_YSZ9Hl z&VrQxxJ@s~b>eQEjoupBx`TiCyb9jzHsNr+q{%aQ?)bx;A{@(yvwkc4*f@yJWANfd z8MjK)mG5T5Y3o8Y0vTyCF5c(ThNrcm5F47uV}-rHl4lR~&V6}Y5i~i?V_dGMRd1hB z@SU#Cx-jNkj3V!oL(6@!)>JqI8wo2Df)VdImdHqQ!1liB;nOWic|cUc@tZf#H=}hI zhDW>Yx=h2J1H6yCQ-len3|zdszMzjT+L82-UW+*4ffre6l!7*G!3^9;@?W`Nx$|W z`DyLWfpJ(&IA80aO50fujd`6veUSKA0gpsaGa9T8h1g^4ZND!VCpj77fCRhZJu3AR zR^ZC8SnAvHKALeEyJTz@DT}&v#4{@elS0nhmZSVtJQObLbHhFkLnb}j9t5EYXLvYS zZ!q)>5J*ZsP`FFVb{%j0Al1LVc_jO_{L7mlo;O>y<+A+*v$(ah0-K^?JH)C}lup%K zZl$Dhn5bp@`Q;}ml3;t&@nQ>^$2ZTjn3s+e9IHcV_xtpUq%KPEk(Kfg2Ito1o#W&8 z(q`4s1Tlny|J;r+BsG$prVY-B@4rc+!Iwt42E76m-s=CnvF@XI1rtWP!VAV ze(ibIU>*xYrJY2~2iIqND0bPo8B?q@8~smnF-%ce3oy!ho>Hks?KN_Q;;>ROdq zAmsNCt%MH4`8QB!Ew?)_Cr-h7@1k5gcUCBuS8_7)?T*_B)R}VolMHWYj$gJ~ZUS{c zu81BnoH(L(uROG4c>(h{J*7|U97jibj97|Jg~i1R!}>kQD9G7Yibs!cdt2R*9H3bT zIPh?C(Lh1u#89(AjM2Ai6OP&;lD%ET-(Q+ZXfk(necqF^A`n_wkE-Z7uyC1QCl@N? z+LxD}XOxi?GE%g5*F>fE7*nk8Pbn>=R`}Q_4y7Vca7m_k&-69Zw4eH>4w1|w>I6)@l0k9{oJB1A#|vuhX)Air*C z*9*p{Nl6;f6O>m3=%*eKUjpmKqA&Kr$`#O}cYOgK@+G$tafu z6IO<+)eP_sm48Kmfgf!wPNrHFN&;L`nbZ*W>D)pvJ$P7SlQO5AIh7m~s3ECQe$3+M zUnANx#clhwH*|?S)kY~_lX)HlY{dOt>Yv~a@LfQTzHI5H=%dM>4O)F9bWA+x!;^pM z35;UPX^c3Ur9CLC%mp1+Bz~9bEMXr>?|a-@ac-uFKVMwHeYKwsP$k$fc`laSDv##! zG4dUv^7u@sX%w2hIA$_yj*mRF=j(HGg$v=jdaj?P*07K!;Fy_Nsqq7XG8uF%3%V-9 z2r@65UOFaKH^$dsm=O5HHFAz9rzgeNTdFTIdDxr8CE zcUgWt$?rH%AXeNAz1se(=ok1lhfSr;kpnyq#Tb&}axzbp+Mj#3FFUqDuNW>Qb5$ z;mmQzozL|IT|bcsM7R4xqCD=E0!MJUPZ73IsDN1Uci8Z6_P(@5#Qf{8e0&h%tOG0F z)gKK;7%c0`?;q`^M=j;xA!IpZ`}4{QVHPjO$)%i{=MK|$np3%St1j~NTgraSRxYo^ z-ImaHQzzk=HHmP@4qZv8HI~f8^{uQsP-Rq%B|mtYnH38lXnwM>(IinX zcq`i%7%cA!NG^@7{ z$WsF<$Czs_7ARjiy5tzFre_!oL}Z?x^&1FetD<_X_%~t~?;(cY-#+NQ89??jYhbu{ zFbs6PA+&ch%QP_-O%#?aAR930N;#YtC$WFnN3P96zDF?BE2F!6KE6Wea{TKMBkwFG z>+ja68;}a=@1K%N4oLGA!R%J8h|nHcu+=?Vm}@y=)IA-MD_UyPmD!n`v(f)gynlU% zSXjV_a`?~>5dy#8DIC~0vo<*~47>F_BO1huGSwdmLH<8TH6CxZo0p57m+FHJ709K( z)-FwM$Vi$Ntv}c5r5S6}aWLi7&Fe8WwW8!JmzD4NoXt?>?w3K zI@h}j7&awAn5DUxx!NyxJvu^+AN`S{Aw(9`;KNOO)@i3`_ko-WdV1C=5QlZBmHL4h z@h5bdPcy?LE^TBgI2At}On(}?0@EnB&EvO$2_CVneF;}g$*F}0Pkn_LH#^(~4{U6{ zd3G+1J)fLJ{lp0fHgiGO+)lqLsr1zcL*9_?)-_t3Q1K;L<>C5iF;E*@*}tfk#gXa^D{jb!avlk_6^ilQ zr#ujaPJ7bSg*-GSQ_ba(mt z#c00c6=T(lj(J#=k9eSN4x8NphwKmk>8@QcU(6-U*HKWZbPMVE5K3fZU$ihgqxR1E*ESp|*$zFYmstDbEvL_t5x;68Vz1ixj*n6ss5`p`L zZ{|<-*+V@4dGh3{^{)^%Lt(l4#t6g6quHF|S{Km2EfuRw&5hR4_UIW3AIS?`NrA zfrl6E7yXc4(_HpSr`8+%ItDNNsqO)OG<+JYj3K^T=6HP3{yKzU?n`E2Zwu(NX2X3M z&+y?FPw~38%aM6}ruB9p9g5xh`TG`%yU~%&@iS>$N!hpX?d0N?yojAy0Eeu2)M;Q) z+}fd{?vB99Oe~AVilN=UP*yXIxV1FE$Q7JjnN=RB4qlmSrG1O?quX#725T@P!}$t_ za^*Y<3%2oLF@zr^k`@ePd?O1bm(Bg@7t_0fG*gMB3~f!gu!aY&0=&=EXje+GLmQ~S zT4i0}$ZI{IiGj7v6?|GYnKIG+9tk4x1)hV1`PXzEiMtVkdFO1?&;6N%&f5(p%R=yG zY-$hKI8XS~y_17Ay=-E_r$kQL0&{c0To`!95EG^nz0JgpjFw8dT0Exy0jn>?$E*49ChtH@=v{!?r+TMAxoM-c z{&ejK4k<_J21ZgO!`;!VMxeHHaFk7nWi)L}*Rmdn<`-ISN=#Voe=V&;D6FrRPQ@bT zkkqiTg#w}@!1=kl=kgO3=I8pTl!BPkvfP%PiKRES0%vJzdEJOK0%@j{S0pBR$>rd0 zho^z$>ElnrDR(;+Qn1vtoLh;qHm)%)Cl(gwcbP|-;DZu~EI24?PKgO^_wZR7BJg`} z56Pn}Ne4QZyFKI;%m=kGN3D@|=E;z3`1{rZjhk)kLgRFjFo+3{aN15iout^n-4}fS zy#6aIR;if8=K<=0c?$iTj*%TiBN&CA8CU)VT^SS zvxw?VMtC{tvxlGhxL@Apq}BXt3)4ON=gNw_uI#pAb@uO-l>kQ3m(CNKus!OiGIOMA zClL*s@m>YfpdXBASAx+%@e_Ja-tygw^)B7b(~JkKXI&%9S5^qDj~yldvcAMxx2w{$ z{i}jX=&{rP^#(rg;VV74`W2C9NaB3gLighM`lWH7LwQc+vdqWT#(byGHc(eY1<70x zRfSXZVayq6~1~E9ymp3ITHJTcRKiJ48{5xXaZRbYXkg@ zcxp^vt2vR|b!qUgla2EE+VUo&CTqv3z#t?mdO}tT`RIO!2hof&Ur+pxw35E)O!A?Z z6SMQ-RgKEbSi%({s!8ET-N%?-0lPICpJn-aQiRvrhz8mOpUk10UNhOhQyeT}FnA~| zVv3}t9N7R($?2I~G= zB4A#iVQ+N6S#DZW&n*|S5PnZC-CL8c5dow+Sn>)eC}<@3jo;Qz>&;%vjcZe=V>@q; zobV}Jp3rj-&Zl`%lTDuq&nL44{lC&2`Y@Hbip1FbJy|NA&1hesJ-CF)EY|m$Go#M2 zOt?S(pp~M_EXBJ4O9M)aP!iq?|16aYxjh>7+jKVxo>iPva6sk}F=2bOA>Zaec2;H? zQm(_nAQ(P8GPOsP|BH%x$2^xLL1Ymw8?G_~ej*ej7UAzYRJO4U>a)5=EdIDmav^x{ z!DrctS<5CIY?_dYo7$J3YclCQeHT{U6)p+yDRnARHR=-nQ-neUf%t-c-4v?8e%;Id zS$yYR{l@b(%F|bkBmo8{KH`DW(gB6*iHD~a4z*&3-H$U>aOkMlUeDOEt*CLWJT2Fy zNK9*A-=JTEy&D9WyEJbv1=soR#fC>;P6c@{J8B0~%O6GG!V|`w{*{Kbj0;inhQ6=A zwD@7?){=RSeaPc`)*1<QJCI2W^tyeP>r~k5mlyy4jMdCEZ5XExx9tsj- z8~v={KT$&G7aV1P5z_o|#oF)h7!x6)mHb^Ti;9Kj%9i-Y;)cFgV=TzESVic$dZC8Q zcohG5cwC`VLV%E6%uEl$%Tq2l!xAGXl;hNL=|8Bus@3WK$z4EA8F_SEelVqdHP4w5 zq2xKMT3f$k_QO2EOH{X_o+&c@=P?@X4dfoKi<^}g{s)n>*z4`(){|-(jDtv2w15cN z;R_{^cq|`b6Tt6XPW(D?dU7%+37AHCOVp=?(+mHd@2FL3_I}o3_Uh3JHyhM1NWM@n zamR!tnvt9&%?IRtt&a>$adbqFF3YY?m|XgEp?=DFEMC&P8TJF{#*aW0kWPh4Rln-Ia^QF@De@Y)pJ|BMC4BRyCK+di_g;&g^eHBIQ{M2zLert52 z%8P4B-za?N$d6SZ;PO+9EA2u87%+0Yvz0-)&(WfQBRmTMCz8{vmA0G#R=FCx*K@Yu z#{mohUT2GK?cKf<-WSycsYw5d6BBGofFBn$EF zP8p_45J{)^*rM52P>o3rWL4geQDXp{NjN&54qcs_uG{}o9>#5+)h3aiQi?lqmFs$q zj<1mWc+t1q<+3J$-HivRbY&oV3bdYlW6bvi2OmEh?;)Rai3thYd(q@MF%q;~M?%z( z6U0G9Z~&I&zXqk3Jkmk>jtA+TK?59n5E6P=DPPS4^6B>1YCI{CgOQzQl#CH)1F^}1 zDnXEsuUslHxt=vY>8|_A82!KwO!*#3QRHF4HYY~XZ&Tg|PS^(%QEI^Uy`g>-8io<% z|4B)4=utoUC#2B#prhJdQJh?j;C}omzmnLu5574)r@FJ@$qi&!t8>~d-Wk+pbic2n z@OUMJuVj^RZ6Pa92mt4zqc7%l(>f*i7c(E(u8n@mNLUU|w*I*IpPJ4PUadx^(@*2!YLVZ0%XB z(}I-Aux@%20_4icRZZ8(OpxlrZaerI{rcD2(iWO z1y9&zEoUIlrS0&B?u5;CLqDVX>;2Tgd;n0QD&Cv;#^!z{0nx4FQyTU85pLKcJ1ODZ zRbN-9#VW(NJKN}OqIy%o3*ZchS%*R3eT~N@yufp(6bUjwVIeP7kaM~Ke)Oc1e;J5DWLu9bt@+eB6=8ETv zLzjW$`QZn~+OOYR!nQXeH&u&6S*@$|O56558YBXGL^b8Y^y+NpcZr=&&b$QXl8WC_ zczksI?zOXO^cbCVp(a3|Z~JhBXrqUIc^R+f-KoLo<@BST4z2vQw{NqfE;sXdtJ6#{ zER$>rf@oAqk*WnQWR={LUBAPu)LwksJdtnO8QaJF5|tLSLTfz$hyT*K7=Tpi>0hh` z6Emsa>RfXgt#-Zql~+E0YgYiXaN~$}O5Gxjc-_qVV~fg%mRBe5Vi<{!=w6<&#khg2 zwL_)tJ+@H?jIamul}KK{By?I_LhdPnOR%-Cs3RX&Q}-!%Kf`vte!rIiB|ot|Z6ALy zcWf{|WSrF|OUMc?EodLD_o&w^lTH^5_DN=de2L*A11+Z!ZhDwLtkbr~H95|6AY!9c zVork*Yi%##60%~4@{uUWRbl(KD20_WCgiU6p+M=>W1=LL(f6e8Wm(8;LR_+j{Xrjj z?H>xCKNMat=eMH=g9k^6G05P|g>o^B&G&2!;^LWjWxT`?BI`bg3xKRqVdrDiL}kIB z2_mV~CKIdNlj6jWb=w3iUrZM}G^yBgVMr~o-ouODYmgP_Ps4Ge#s2{jukg99=L}9p z-oN&L@EM+5`?NuRB1HNTnz70rTJ}aWrLm}lOZ@y*o%nP84*43J=2HJ*T=OeDncjrJ z!T=`W^_Ev+4|$a&)iJ#ASo3WqUgOg3J1n$kG51-lTj+PR2E`OP7UtCDRHi4VErv2^ zI_sSG1aZzE%uEF>PHHHU@&^ic8BaQ;lvx&0*<7rY#K9Y zN53PB)E>#YMR`{|;s+^mD3I&|BGr1NfnHh7_fv0^w~+_zW&>QBozS2AUN~qg6KTFg zVh;mJf9JL;Hca%dn4^@fa)S9zMkLcUsoo-6K_XJMQ(|ekPvsQjyxjLyFlNr)a!1jl zE+$yen0q0RE#h-5o)~n1DoXVQoj;_n3orfJqD^!oES$*Oi0}H8sC{tZS7G(EeH6*( zvFrD1;|l7-{xKs+WVPm&&puE}1_oj)QKt!GS*Fu7E{|o3cfiEqiMtgN+7^5ldC7y` zBN2mu65xJLmLu;O#32a_E#|RNoyeP6^MnnaxU#UP8s3cWcie|C`Vk`;Mq=>6;&-vZ z2SRfy=Ezxu)mL9stG9Vw>pfte%G4t1GhgDCup&a6kfTf7GYe**3l7TB9)t-&;G&K% z;v4BCDw05L`spZ-3H@Ha+SDAjljDFFYmJw6+3w%Y;OlUb3%2*s%>_t!ttkcv$UO#M zb;j9_>1|@6_eMlrpwGNNSm+XV&WPf6g6}-4sDs{iMOn$$3t~j=Q4K!y+UYoE!EF-S z{p6vIPa6tmB+!YDq0{LXM6a%CS^gc+h%?0@5HC;JkSg&V`0%r$#y>?Kpdt8L1zX(< zp27dVzZu{nxLF|iquzz~=j9A#R&=m1tYhRw0cNPXvx>Z^Obc8H!^YQ$?IIwbDOYXG z>pe`CllViRO_;oRE70m1cQBTKz^n7u%E|lHlGWHn7hzOAbZFdc@q;;v@T1V@m*&we&mH`bFkzA7G=a`n%J(SO>As z1jm)9D(CcKW{k;n1{`t?8d&THakg5BfLkX7W5PsmQl|2#Ez5k5ijqh8$EE8Tca|`E zzA9zcAJjG#O^AwWKG>W$3~<0X0^*36RR|33M^6&=+XRl*kzG=&;04tR&QT74Am(jd z1*cI8g*Bu;vH5V@%Dpj8BK#nDO!L{Ok%n8S0#+w%NI+m6u!baS~yWdCa&NKu`#xg$QZgbkgvG(JI2|P)PB*~<=5Ge2$&@O zrGY0IO#G`66SZ`;sRA2m6NRgDqRBx+f|S#G!y5-i_~@LQ60Il3eZp;{oUa9#&)WuL zYEIs{+HQ1bUHT1)F4QE7VD5y=D7xNmsI7V9Ah0T+?9a?Bl>n|Y<h8GY%Cs=PF_SY1=4cUxjD zRNud4)^(OKUM7+AKYT@@X}s3Et#y>6>gyl)!c_e@erRAJhV^YgCfSFebYRhK^!UwU za@|FS1nU~ls~;WquN1}+43of<_>U>IryjNXpPyMeY&UoMkhJtDYuj?yi?6&Sgy=Ql z3N)?FJFty4TU~CQAFYhr#Dx-yQor&RAlkJ3Q0AYF@-ZLJ*b0 zETn=yJb4AX0QhDH0j?e-08Rn$RoE^&%y#?lt-|CT$)d)&m`jHu>I!t_9$s;LrmX9q zt3!)tstZE64TDIf_f=HzLY6VEIZyO+-mMg;&@?Qp=}8O>M~oD}YoQD)XBs^>(k znVIv@Z~O*|9weeJAm17+9L%0js^t89x*GD9 ze_o@^zb+1LGzWqoxgB@d&EIxVPBEHj%(c(7yZmKU8dA5b{AETghpHaIGHC z-^p6-dx=5O?s?r4kRQvBs6PS7%Qo#(c%pF2&}%26Yh3p7(eA6sQa`I_))|CBj~6 z$5n;R=}JhYc}*?!Ht^kd>yLMkm@HWd903Z(_)2tb%0JcvR_2UmCtctY5uNWK)NGrr zhYM!L{XCapihB--5DDntXGTO(iU#W}wugzUO74e)Pmhzn&R(|l^^cqJzCXZnRMKOA ziml%wwxs8c{KvtQ3BhGOQkRpTVi4)Z#!S>h#7^;AVyeQW#J_wwj?k}FtbWfu*#kR= zqfbs={R`r5K+}d-;5f9|#c&yW@inbnT*}_Z{&G4)Q9uDm==zH#W`i+oYUvg&Rvr)l zsjnJk312-z|7PFI(*~_i*$heOwDt|0&2d7*`9mSF7;+3F9!`L#=Xlzd@e)dNt8Qm+ zG`aKWYUmd4y*TAl6;!i;&lzsR)o&o?j!6C>o8=#k9}ux|_HRm8LTp{;fu;$+vYC?V zPrJV`O-$}2fg$Mhg==#V;E+u&m82vhMu6YgD-_rrQeM;Fdgeuj(`Q^t5Ka zXCLG{=vo6cBjAqeU+1n`QeiseNoK|qF9eO0bGavQTMOVXe`)$;po3BRWWZa}|3jq@ zc@02Ddob{)_49>{0vymLQU~ZnSmf5!+$^W8{J^`BtV{p>Hf+wX+|lDB%8b#Y*pz|M zRjWX0Ty(Ilnkb?2Uw~Rru0gMKTm&)0!pz22fa&vHy1)5_K?ovx zNXFhn>2hx&4lwKF#Vn9}E&}v+$yv7B`1v&>LoABvNzLUxREx`WUR{|pu0n>jP9`lFh=)`v_?A6P~JV)BViI z*f=b;9%u{f8xj(o%GSsUg3xO>CMrjNYaTVqX5o2CwBYlEj9g?p9;N{AZzci)qL7GD z7*@KdsSmC>6><)-q2f}mkh&(WJas_5B|DK6^#^Ir>f50;VBpQNLkpfP7H`~>Z=7Us zFX-8`XJS%P!f(?zT&o5Yp@6Eh4w*VYd*v7bk2;CClHI|iE1dk=qd0~yh?n>UWzTev zyF7%t6D^4&4XA`DdASL7b%{`Vy_#_U49wFX{92#hKRZr`Z#O-d+=eJ^YEM}xZ8>AJ z@y$J8v5}&|QUty$R*&~Cy!cIO&zrLO`asRYB7J%(QXSm=9q|-IE@t^kpo;_#we^%J zs9O|WXE0@i3j}RD&caFve*R7#1z=VfYxeK`X0JYcv7mq^Ag&c!tG@bV%tYT1MV*IB z373O<*;DLuH+!-cIIgE)-y|LTkbhGs-1SuY(bJKs`-Ur!p8!E{qg~yo+CTdI)+Ir$**+hcuZ!`~ z#Apu@T-iU{S`e} zNnrB~I%UP{e@G3yB3T(k{A=`d-Yug;+jK&TGVyXH4MP4T3Z2?{slDm%6i8s>CM3BO zedz<^7>^)|o1>;KzK33~;nsckwQNwX7A zn0AHqebU`55NcU%#_eJY`}+W|D@0(wm4?O=ETQg``^~ok=SRim!}0p{h2T>-Mr{$1qA<|2qwF6Mq+Lx_ul;HP0x|oNvA@HdneIsBx-Xt z-oFT8T`TPSVElZmP4bp`uGCf~X>bO1jAcm8>}rx{V<}=uc6r-q4RN=_0N8Rt(VzoW zUr;1}`>We~tNCEfMvhJAE6#$F^6@P3fDj=I6Ah2-r7Hk^$G|(evU>KDH0PyTz$;kX zkL1soumaNF=$OH8GYXy*>5SCkWEio0tH>IYbDn&59k4l=XKQA2-_4%faLIM$1LJSQ z_RnIoEW!cHv&Np4tq-NmZ;IQ+{k;yFZ+-~YhLhC0+Fx{0>arK074|v2c1Z&)0Zy`~ zMsistst-M3|JYb0=TYaAc2&pR8Zw33i2##)|I%}T4jy8xLwWDw?~e^kvelNw5@ZcK zZ(v}cH@Pl=xbBQWr!djcc}+<$er!FeNeW38EJfCV;a3>DTNd{0`_l(A9z;)OYBH4) z)@}tG@IlacIxJvM87Op_f0OyMvpEGS%Y?p~;-hHrRfKtpl~Rpt6?7|Zo=A^{HdrlU za4`CY0|iO1;6&@)c$3ZN8}DgI1Rw;-n@x)UudP_TsRQe0YZg`C{z)yAv0yr7NX(O*I5!_9xA5CVhWQes}5Ro%B^GlGmD$44#5Q)_!KUTP3Rh17UL2o>Vbic$1}7aH=2e40)6?S zlyE*`cTzXBI1tafoIY1b`epHRD=HF@LWzSxCiPI(ss{7RyvOndG(N*PybEU#yPD3FG$9$WeUtEmsqNN2g{!V&7AFGiZuy(Ut=w@*L z6z;bfB&Y2&HfauiyXBc#*Yxwcfx&4^!1$JFXQIjAyI6CE2~27UBS z|K#T>!oCbW%L=JX)snp2H&vNXN4&x5{#qCe9Uv5hJ%QcXxhH!!A3tR^g!Lv00{+l^ zD#Q<^E#~0lt9YZk3%O3AS}hHPf*%)Se@wz={Mdf2i@%mnq#%NneZ+vY*TC}2gCJ2$ z-qt-WL==gqWZWDBNsd7&)EGPRRmm?gImBi=4(HR(U|B`DlzYL)6f{9cl(~^e}{e)qNv~OMYqXlE^J*4w=7Mjge)l*(9k|47AWc0 z7$VaFSuGjmKV>hE-RYhd-=wz`x+-)w;~jXfZ~mn{5NbK_>Fj35LzU;0$glKHOf93_ zhpEplc7qqb&Da@PY=bLKzKCI4j%o4e6NscbZ_SKUcNL@-5C-KE_n9<@Vr91p4;HbI zD>y|2<=aQ<4$henND=$`A+#AU3|bw1w-v?j+ECv;io&X?U*fWV9Z&ud6o|P~{IR2V zC8Xt|kQMUMo0od$XVf~lmRI$am9QSy z>nv#ODm8hVY2o2ky{Fg!_Pub({|Je0Hvr$^TglYEGQ~ji-~%Qiuj-zM$U!1kP?PQr zS3i~I2j6#iOzY(L>^coca9yk{D-ca=(aS@a;BzUUD1J>_PND}P?y8x&?U<=nMa+h5yfbirn5R;wzU#>M#G>*TN0KOw zXqEhzdVimP_4EaF5;4n>VkDzBlDpGf+L{W>S6;FfH&K}pDIi%l&)puoEHZw|p}DPG zx-Axacq8&x!-r5zmH^M_ixfK+^(qm(SnCyJF(OXhH)n0Zw>k^uto{V*elKJAkJq$$*vE5&NQj%0ZV+VRI9Z>dX2MXE8p#f+k46A zAjZ@OatBK`Wqgz^lgk4&O&T<0hy+rdgCMcysTp?@I+E(gBS-t$r^Xlx{q#(|zCIV{ zT|=Zsfc>tokvjcar}&jU;*yBf&^fTFSSeif>cjBJA+5cMCsr-7{B$LENFI~0CUX)D z0Qd`B%*EI|dr=Fdj=Zz2`r3OLC8A)h{T-9=>Y}vDw5FnhImxU6ZsFTkY=z;7ld+-$&i&{0Ys9!;ce);De?VRVUTK)tmU*ea*gagMNV6u?nFo0P)yP_8t#GQ#%*RI z?|?J|ijHK5?-R;f)dj@Se5IJYbW1i7+q~1Bh_QR8L%fjCqvMNmQz0Z5zN20o;+>vf z#Vl-`m}*iZ&zy38`mX#WYEjsrH-qS<2$QEnMh>}O6YmU5P!m{xPdj1pb3$UWXciaqR+l&FQZ@BhSqfz7VwaeW)xp-Ea)z#_ zm-C#?dI`DOY*2^msdUcd@s&l6W)YOGZf&)I1S>peuT)trVqHoFp0!Z*+!|}gqTlc69WXdnbm-lNZ>1=#EV4;73 zLi*z)$lfc%*UHtgi+y6b3DKAYMi>infQ7Ebj~IQWeq~i(PzBLS?OG9?2+J<-6x3Soj*9@DEpST2Uda(tRfm~qR>ngs8e0Z7 zwsPgrJN3WkwTLl<{A+s@&aIj)tp zUqvS6(eb;H2+Jz$<}nO1jO5Bf#IEm-Y1^f&`J&XE($E>9Ynh0+=f`QVC{=7rI2IZn zcr-622|O>s7*+olLV>-eKc2!De0}~mp!*lkph!d82;w3Mc#Qva{1QV#MjeYD2)`10 zV40ozAYvedlTu`q^m|o#MnTaSUq@N3UK=aqVCKsgMr2qkCYsvD%QN%+mJH~Qo{pX0 zR5DuNU?py*lm>P>3Bp}`iUYkRCC(4b;j;#of6Ebo9}+dPMx zNEgVL&^|cq?iThRo+JnHPQpMZAd9EKi4DTjFgl5DM0~~_{O%dYyd4XeePLp%akzo` z6kXp*GG$ z&ub~2DyaV^_Sd>Js3xZ#kkOU7>q&f`Mv|0^*&eTaK0-UoK*r~o&lx8-q4zR3MqBgD z7`Hh~ofopPq{U=#Xl_*8i7cV*D>evyV83hUR?Al>(D~?qP^`Hvn=!U<&uKlisGoZJ z;PnJEH7nLkbf+k^k-|el?c78iiDL*b7k+a90gPNH37_6aU{EMpz>T*~ZmQPtWGz%n z;C-S{h(;yfph+e|8a#ADqxh`kTk0lRXSr)|U^ufabF2BCD1xC$ky!{D+LCjE@6`Km z)KCS7mgm@6*tiBwgrl)b);DR`+IjJ=rYaL58?9^R;JAQZXw%co*k~g(^%bK2d?b9K zGQ;}k)r~U^I`je5TIn&m9G@d`^y`SIf<$}8aoPssx;iGe{x9ok_F{uf70k^k&$3dU zb45$}%aK&-I(!_fj9GWCaK7+e=l?-44jS`g%`i9nD6CQLBj7~%u(%h7z4O@bDOss0x1G;jtL*<~ScO zPxE|myI3+s?zDStGons2q%JSNmv|u%r&0HrrPz{7ezdPnWIPnNV+{grfz*QF*AV4y zfCBYj){17@K?kub75EDr|1`}r1ORJhDaDlTtwV324bgg+-omh0f;E*DB)ACJ=}B8j zS|9<-^(j+q*pse}0x|TCrA8=Sqy^HnHhOK3f&BsdVmRk7N=o(Vu-YTk#Ps5+s7r^_ zd_#vr(~Y0;1L3b~nB1(j6Jo92!9v+Ig1A&wHeur2-&1P7DkAfAadb_NX zY-Op>&S}pAljy#kTa^`~9?m>&;5Xw9G{s7yiMrjbpIDOzt?uV(U2oG%9r&4e9~e=N zqQK8rMPwY#>6VqBPmM{mbZiqsH}kdGxwsmZm^S(4j&P93NtRP{l*aA+I70${yYges z57_2O_7s2}Sg79aN{tuTZ6jB??}{*VSfH3;{>zY^OaQ^|H9P$1cP+!eyPPD0-q86{ zoQz!UH20(_%MjvT=J)k-+ws=}$n(;bGv)3MIfnYV48(dj(+59@@l>z2^badewl{Xq zaYc8o(4R1!*5%1$xte|JSt2+Js@Ccim9s1V2r+@|+4(2}7~eT+Yp>2z0^f}R`{0&| z%Z2%+Ffw`HPUUJBlGC}BeNno^^`MdI@%7VBNc0{oVQ9Gcp}kj^wQdk%>fWR!oGE zPSSUWU)8p-nm{Q7dN?k}z1wuw%O_765@J62VxtG5DRazm=3BN^eYS_V4Pi0IhkeXU zB5m0sJYDF;FCUt%hB381ig+XWBe1z}9M6nCr)|Ay>vU*CO(UHZkX8BW-o%9WHI`j( zG`S|t8egGXwp8kWV`ZSlGClg4QlRR_4x`-+E8%-viN^7@T-+FyrTuhl!y6%iX1%r?Lq2iq+<{VdfT;KH8@ z$lnMzsdH*Hx=4D-;u%5BS9ZA?cir1JKvVEP4XBbq*634Fbw*^OMG^E61sQ=a#f<-9Oj93hrf*+BPHNg?D`N zN9o%=tk1-RoKI1|ye8k%A1HvmFb>;OB$!Sd=%OOnX_(KP#(T?+bRtBiM!H3Cn(BN^ zm^9QOe#b{BI1RX(i}X($(v$ze_5mD(w!pBPaB+Eg9%v@m2(<3>1nMX1Ewtw8Ub^+C z0J#_D!vqw~UkCS+66#opk?h2p_v@{u9P_K4&oeVm_Kwp#I|J56_?Va{SswIi@uBhI z*ktOLya?d!AzArsI0Hz%jD{OlUINzJ{ON@iU{(9E(YRgu^8-4^N!E5Y?+%^hysnB2 zgo_2j?Y!#lXWDc?`3;;BMW;uvZjWX z5YPwcU5D3D-7WW zAR2?Oct*Y{6gxZaYYQQ32r%OdF!139v+`P4hJPa604evcNPCz6lIciP13x^1j*Cw zNA%JASk3#-EAvHsR7j#-@t`4%rda%F#F{a!dztf`P&^%WPXJn0;Qm#l(pZ{_1{MTwKSS4M>xe`6#zr<`H!@b!UwzDhM+emfl11ea{T^yCy z3k!X?@cKG0E$z3YUf(pWyj`3c*d6G@j?aRRbRazI_ytpVn2{!n^;_jsV5T0yLC2VN z=ybBZ7K0M~dXQlZZ;o%jDdtAhwuX{*?V$fa6$ep{QPbBfs@3M^Fe;J)nQp(ff)*o) z=5wmEB7QmLSLXCbksRaWQ2?G?QNs0ufE+|JNiwPSQP4cVsz3#U9)$G$!FpoUG6#0t z1VgAHHjD2A>we62^km{g-y^X$BQ|xjW@qZw)nwe`z(hO+A2`%`B0FCWk@b?=YUY!U zWSL8RVq)KoB87v>oI2De^W)4p#+V9~U|5+|R$&Dke?XZ&biJz4<^d(5$f1(oDRGRjE zGkZXF&odo}13R*{^Lo1famwE6VCT_mAAyTTeQKbo^lrY#B{GooSlIS%EB1kaBbMk$ zjfq5JSUid*GbwxJNZZ|#=B2idsBS5OVG)J<5W|nNslC2!78kKQ7YHw#yn?X>ZH9s_ ze0S}y<+#FCsj{UdaoJ=IdBAPINx)ROh?U@Q1Cn&6HD?ejmovwE5qru&EU@SPRR}x)a#43URRv3RK z_fk{`=o2*x^hQc_+2R%w63VduVZ=V47H`o6G~z7;8hER6+bHBxGEyX5583Ki-R;PR zfOyQ0eP8~PdPdUqH2mLVlN~@8pVAz(7^MD}U1W=}?1ZQmwTDVTd8oOO`m55mE1>6+ z?$yO<1yFEZPEBo{#?&OsCP&n$nP}rKQNR>Nq0|ac~rH=ZK z%fQ$Jy?XV^5h!bJP6;PQ3Gb+;<#B0;Lds7h^L!Upk_wxFDkY^tY4Gw>5@i_u-SfaN zz)0dH%*4N@JY+@(@8?bRIY3JxH(8=S!8*<*{J+fe_h5s*p1 zLXhm?ewQ`i^@74`wU69vs>&|3B&z!+fsmyF7ATVVDQVK0*`0@ygtLbf5mg@=egyH2 z>0j`}3&-lL`s<(`+8YT+v%805yt_HrQ>oC2cAH$0{Fuf0!01Q9GX>fJtSMl^lq=k*vgt8koaNydGRM4bYL=WkY@w;HnZSf^Btcx>)$tDc(3@1E@8 ztQJFw!mqLsJKPZe&kFW8t$?Jhf#@CP+5@buntkb#(o(0*2ppOnpzY**$)L0iEk0)a z>L>?y{rrm9(!ckFBFFGnc|#{$Cx%A9`FA-Pe1o|4sk27ib8a^?!gvV+THyk{sVBXr zsvTT5U%z?tIMMygcoHaO-)FSw{bLP9x)DlRd)*}`BKB64QJkQQQ(c6}YssD3T`=xK zVx$v!@D*w>v*}yS+i%`lBa8?Ta17!j`_T2nw+a3Fz(F*-hTVlW>-}y{dU((%Pewll z1^;lF$@(fnC;z`Bc-SOxe5K19&-U*nv!-O~K$6|x2f`NuL?dXS4w3rLeERT#0Cb62 zrRWVJ4=*p@)=fK!1)zs~6VT+BFRJ)0;nTLSWnYn6kG423Hl1!VEB?}49F)79SO2If z1*Na`F%7QnKVeJ;linmlk>3KEM?2PJ%%uTy&9}*yh0`?P^UP)&->$TPc zkQg3oY>L@qJX7i)#3%>^fg=wlB>#+_z{u<8Q6A<;y{B#H2Vb3@-%g1JFe`upohtl5 z`_CeK70+?6gSwh?@@G?1Q-RsoQzCv>f@n?qSaplu_(JY*qK|Phu7%rTv;X*Qh0Wq8 z2mK=|&loiSF1=#u-F@>}54>kJpX~TQq+{`g9To7mXF( z{@*$A6DhJu%V))UcjWu4ItV;o#XUU8K%w&!Ah*#B{TiQ94K}FN!Fv8xI9{^e2DK3$ zB@B>Mx>_=x%%17+Oy36D*F9?&*Zp)D{pI_Q_4`!8aMT?Q_CAGE-T=xx6dgf(2IdfJdV@_Et=v<}MsMa*KSVg@Il#-PaD zPV{f0Ui<()o7`(Vd7B`1rxufN^3>hJDk=nk>_eJ6?J-d>*0erM0EssOuu*lwW52)@D@_p>%51f#9qpVcVHpqTpslpVAUk7){Mqx6sX023>q&` zCNlLnqHoL?^2vLcCeyqWb?Dy&2xsaA-vOI;mdJF4=_Sze zR)VUUk4oknVt2y`!ic;TTCCp&%$?|`P&lu@H>qO|ww@l6G%7^znUf@*_uaPO`=p(~ zDtnMDn3X2o@9%MH0>uM8jl$n zlnWq;A0W?>9BIkAb+0|NyS$+-_!zOPtS~YAx|aNyj~3?gLo&DQR@T;3?3ZkZFQ(rB z!GUYd0vE}34bAX3niH(&HXa4%JoYx-ALHGk4Z^Zcy*7k1LjszcBCIG&fe<=p0s z-AN#PEI39wB(O4iLTzf?;zdfKi}dV^pST>+pcj-oYds4_vW~&-!yVkU7WzIA|8%8n zz*OS$SJ#0=!keI7oJywJ)(d9~Ev+w2UHD)@6-spQv<)`#Syls-k9OzKMK#%Q%P#B! zO=U>PdDZ;-{Vk7PcMik?s_)3GvsLg-I*{lwqje=UkRfig?O^!oKE-b$A7u@OzgGA* zPVxjzBP=GkN2aJa31CA8HmkG|;CDlYdx zuT|d8Z7}wjdS4@6+=S^YUF1)}SyP5Skbz(lAP{1`m+|<`TB$OL*1fAPlhdS;#|Pwy?j!Ph!+q7`G&oslYvDPp+Xfr%PAJD z?b;;08y-Ri*byAk^Z)ULcBDX_Ybqe8r|>Ypq{r9#;V$`vvV9=v3P+{5{$=6WU|T9} znv1v@kOoy>l-jC*DLQ?A8syk%^wQP#17&x7}r8MDl*Ibq_*#Xt8(sxr-rwV1nlL0!+&kp$=LsD+z_muO*)zN#q%NJQD0|P zPAi3Y#KN#zmy8T0&Out12w46O`q!IVIg^hewlaPjN6JmEdX5$=@0ux}(O7#RF(c(n zDtKitFLL1n8<0Ph%NiehxNnbc`*n%TMaecbfp)GWllnJYs*k3MF1Kg?$t2+?AXdy)WeAfI27igXKQJNR-IP% zQIn%}w;ZgwsJ&dt0;v)oeOMX^YJ*Ut7F#-BM5Ixv*bm!6p2Aq8tiU8aqlM!giR7go zOM5!%P=h)64oh>O5;!eEgG>O)AdN)T^m@q}s~^gGOtJJ{^4^$tGJ!yJhnAZDe^2HB z+UwwD9pC|zTSpHyYh;Jb!$8uC?l2QOfkQA4a_J<*g=AsuC;PeVm^6xaWnYHc7j^JFBt^iC80MxlG4suT9Wnpw1XZ&eg@*AkIa z-MLU73Me=nSo=tt5EPcD!^yJaHUrh_;&0%nT1<64M%{)J=89Zt=eoC-&S`Nc{Rh+GF8HiCvJ{|Ou`d?QYEvLK|#jB|4Ef%SV9#}t$IPS)+D0Eto;zFk@O)}OzJ zKKlV21Fw+hL7*F#=kWhhHDN)59+32{nJoOHu(rNYZ^qWO)gONoTmK-~+)}A- ze1T`B)-4G+-EdslTmdd2<>~7NYNyoTcP8|v$3N)|Q-z^+`{%D+3s72EC*xydF%VOk z`#2&>nmf;9c=5|=XDO_{%U>MV71J9}hFn2uniWa#bKaxl1%2{?<-==|u>W#7xgiwC zx8tGDr04oMzuEI%+s?gYXs6$)3W`6Ku7iV>e1-V>eHQlOL89g{364 z>%@S)ut9AbYJEc{W#&(0w05q?s?i(D{Lj4>1k{U`zHs>!A5$T|YKH>pMq?h)Fv=Q! zjd*SJ$Pm#bf)6IueDt9A6PD7LF6>j!&`>bdm^D$Gi$v5#eG(CG*g z?OGRyq?PuvEeD4Imvs~~3@{Wl?C*c$DxNx?9Plgl2xOj6pxb|7c4QRpGtety_Gy{W zw`q*Mo#k7bfrgah!QddUoRb~3TDN<*X>?0N@LHU{Qhx06%vkzH7Yy2$I_-MT8p_rK zwD!TkCGN-~x(5zeuMBVk?1`z}sY0s|ohZ0(e4U~G&ig$(a#8QYBoy>Jd6krWn1=7ahQrF|r@ zJZiKTO;*KFQ@Pec=XeJerUb%QQm53x1eH=T>C6F}{j^<>ObzzAR?RfNF#yUbmKg_=l!{sY|v8H$UjoMe=rX%hD+F;j5aTfgDGQ17Gn@ zWfJi?UME|P9E_vpOGpKp#^Bms3(S}yaq)LDV6s#-VnIKn9%IDa*;Jko%5r^{KI~%m z;FNRoNJU#A-xI@re4fO9Sge@!tsxXI>t4$125@Yak1o`6iG zk;@btJnGA}=*Y$l>TMgD$CjQ6m#>yGZs{t&P);ay5;kx&akUMbKe$1@G(OOom%Y_``niV}usFHQI2d zQy(tuY#6Ar#fX?>&jB$bLUVAkFv!^Nr#=@_R}SyVqv_4uC-ILAKN?&IZRd1NdAyOC z*h~6SKYgU4fCi{=&boq%A)mrala~p@Ym4L<&p;njEmY%Gi4iccu_^U?DCZ(lULz+7sk9oP`!zl7i|OUj)5j|OH9bZJu)?MA<}I=iw|gGNMToo1;$7wF%$ONE z4BY+RL&rUVt=8B_56z+W93>m+1T|?Bs;8*=m9+q~L)u}ZIz3&zy+lzHAwTzk6#2{h z8-tXGZ#_O5iHqBJ)sqsMewH_;z@>=OY0q~!o|)Je;-8tb%j&J1%}AdYAp2!`hM;G> z9>t6D4u4x-Fr^~B;1^6x2`)I&?fFqdZ29&2_OQprQN~8mjCE52#ORz*=u|;`kBz7O zn0f?c4x$=%el-H+?<5(EV#+0qCuiHYdyqBXbezlPgE*VO%cCRs17@M%V=u*NHlSp1q8|H^vekZ4k88qNMGW%M1|6hE@%Xc- zMehc5R;1EEjK6uZTA-wtO~b!&YUk()`Kgk3IcHXhBL|PDPqS3qe8Z7!6NLen?&H>` zr_9&ebJaW`i5zT_XR%xl>a};2S0YYE)DPvG=h|6%x#PAyeAh`Z`;wgCyhXY})$bIh zs2#iVAgDz1K^&N+S|FyyP;wh5D3bZLfuUm@Hv3H|+81pQq4e^NYo4QekwbFhfj`$F z7X$uk5<~Vyg6iqdkA4t>zH3-@>t!XX@0M=q9c)U6)MOY|CzuA{muS4O$Z~I8(6q)( zpn0S^DP5rnrJb5$8QPRBigPW+CV8Qtt*-Kdpj|T6Q(~FmtOp5dJe??Ng-dfhs zb3V&L?uWYxqht1_@T>dp?(KLVyLm_z{sDAscl2|=?7m5=`fU^J*ADek2L zTD#HUVds8)7NobN(X#^<6I*@=}aSDvh*r^xDdB<16K1M^DIHP zK4uP|20j4zQeN-4SW23p*NIrq!q|{xkOL;8ys3)%69)#cs(oRY=%OPCopJu#0P#P5 z9&Cdih|YMjJs|Sa@xOa}x(h)T@uY0d*}Q1Trv6k}g2`H3TFrcDn<|_7Z3ZgL)`L9a z7q+9r^9DOqR>z0?h}OYScYQi>UwaMsnJMg+71f!GeH^~0o#m63dniH6DX*PW=iM-L z!TN50HQ^^{Ox}9ExWIh-UY3k%nd+hXg?d|eEcnQlvOsk-E#rltu{d4YfVp6eoBW4e zansu2OIh3*<3D8zcEuwlBsJNRzH2(Ig6^a#CF>-k}c4(|M|5do>9dR62u|A0e;Ep5ZXJ(8cO22Q`j z@Hn+9k*VSDMd32^$zaY5c(=#EG~felC2^CNO@b%F{E@aQ#e~Z+)+udrF-7h?Y|ytW zieVm|d9#Ca^On!|@e!IU(cHD!1I-5K#QwU5g-bz_ZnBmF313TsB7i%0+U~5xD~vcq zt{sa>`Qn(X#?+gY_&TG0t>0!(+OPZUMb)ZQcdjBF<2<_TL6l|qRacf&(b&}2SeuiC z9voa7&J>a!y<<0Rt`-M7Pv==EetG7&Nf;uiogGaYd=EO~TvOQ1MOstijTbQ|_0e*G zrGmwAL{&0!+;RN*Pk)z71gpj>MGmu%{|f?0Ye9C?o|;0C149Q4x60iBx(+Mp9H9sz zOh%=z%?O9cic}Owx>9d)ZBQ~F%i|kJ{0LoqxI{*#_bg4>05^*<&BLX4JU3ia@+LTq zO=3mVYV+{a`%RzcFc(c|7!X)mKdGIId6%8_fpaQqCd=og%1IVmXK?BQqWs!6B8sBN zM?x6#)bM0wC19n&?Hi`3tLpv^3p!2*I6a!c;z}I+#oFz>em6v_s1U?{*VSB)bb&55_J%-_0DtdPEr#UtVQphZokyXH8)zyo_Onv4#rao<8ITmJ4MJ?Rk_mpT&x zRGlfEADs)4%xP(Lja03}?@qhaNCUGZ8|}F5IQS0=TL>fwrS8|S{A(uUxSv>aIzNTd zN4{w@x_;}{EhLo_oKrB9dN+HrPz}IDElgtm)j<*!JVW4B1oG5O*&(#Wv?#q&sdRO6 zpVp41k3N`l(NM58#ajV5sBk)!7f4hn$Tb!<8V^P99@L&-DYai6y_O`G@|)98T}TD( znbBX!j0WvtyLlAsV!30IN~T*Qh3vdQIZF&@3x$Vzi*|&9cUo>U2npS{qN5dIJpW9; zlU*7%e~YVKKc*@(U3J<>hf1^6B!Y(SwE{LM2>hU04#jHpDOzq)-)I#+C5t^8-PBX`58-6@Xai62)K zdoTK=4~c-k&UKu#ym`+iDwc5gab6F7F{+;rTH`-Tr3sD@i8N>1BWh>Efc7wd<*z_5 z?B$8knZW27;eTUzdFssQokU{#4ddI;gCe z`I^Uc3bOxI&nF59H-XC3&TOk;hgDnv@yXEJ`)@%VJh_v;U0(wRg%>Ds<|2W~yb@Od z;lEpc_U+Olp<({@uX22(z=Oo7TguhEqKq^*N*<~f%D$lUsa0*c^nS{t z-x;KUHbYKlmX$Hw@?YY?b$x)Y@l9P1`Jp@NaQAFnEQhtV<7=3JQdq8R3FY%w>TIRl zCHJb6Y}W+c)WYN6T+2%-CZcWnpd+PMpDBm0dh@QmYT@2;B12<6tNYO-Z5hl$%o~ly z{b=#Y87mBEl5DlTxYs;-Xrm^bo_%4N`#mGMV4y-kw!>hH{U>c?2neVGF9!K(4C` z?Vle3qKCFqGF)XSPvb56JA)y;35u1k-5FGY#HYZeiuV<7R$vgWL^2vh8CH4O|A0bL zJdovv@|bBL+RILfs%`)h)mu~DDwy}L-A`)!J!!B7kcE(r^~MkXUfcHv43V`XAEp+1 zud6?~VllFill#u%@!`FYQ1iF|eF~xPb|ID>(7PC2RGS?IGtoDI1aZo|pmTXnC^A<$ zF7e|R-+z`N3lyZ5JbNlH{j{PVMGMn-aBRKdaMS5Cd)Q&aJ(xPVTG=f|zrpBpyTG8l>Z@t zU(Cf-3Odl`wzC`Rq5S@2(~gsV)3rTdgJVE4HXGy7-O=x}S?UAHTRM)tt9YQydBbiM zmT*h)PKtMt>x4Pt@5N|bEXHS{^6KA4PAJW6eg*W40o~>8acHpT)G{O32z7BNM zKWx^O8S3njZQscec^36@{E?_(2%kl3Hs4m?puAbaat`GSqMtCtpbS7{@BofR#c!zk zF3A<#dv%uQLA{-=5vdd=^Bd~1JWAVqeO(lcvu%b5q!ciB)nQ76O`DIJu*MnMDJ2{1)# zBKki6KmM>`(5!PvDfHi!=`a#)o{mg4x|C%yLIK#<%Dnf@weK3NObi%Vj4nLmBgfUe zF`Ykk7mdSDp))udd1E>A(6-7)`{u79#Vq`OE7i*B);r8h+%hNbYH>#*3Jy~LDCe1W%c`IZ8Wt-q0|FC$2!_v4)&>R+-CGXW?R@*z zKx_HiJLJ+YJpV}3_R_D>zXK=O6U+I2SG=1GedLTofgOR+E%X7|r=Wr8&+?k!B7z?S zag)ED6eI`7;ep;3l4Fu0s^SS#o|NrPZHl;HSiQA{@#+3jMn4b`z^)S3`XRq?`MX7v zVnf%mcP-BAQLksZrex6d?Q8DELvjL`wKfdkzrMxH%ncc?!yS7UTPHzgxWiSx(+kR<(zQdBo#aZl8 zs-o3^hp!m3XAwvG7U$!IcauUvpXbF0z_7W?x{5kqHn3a8k_YTow)vL&_?h#1OQ&%D z&^?$T3|Q7PvgnL|FY7Ivrrz}l$|l|SBM!b0W+n>Jc zP%a8Qo&UMqb~-ew-YoSRt-iJATU(0}3Vs8yeuk@>v%FTcEy2a&bU?t;+#lA?QO1VW zASW8Kh*B9ah^1t`)lOm+RqgN9_C5FsqiE0ra=-%5*J}BHj{hShfGP0b?N-C%N)S;F zCXHMhGY#IL?VhV0iYifbd3ZA~&8vSJ7_Dadz-lbb;BlHmlQL(+{-dQ}jn~sYX$w~!!`rDf(mNT}c!~q{&PYzSP8}^diGqMP2rrcv^)PK7)E+!d3NN|aR{+PlK zColjeUU+n;nmd70Jl>B{r|Cpny-nOKI|&cJuY@AL!l9sb3iwY@b_({xd&-Vap2M)x zZ+A|`qJxZrG-8`hQ2BP5QXDXA@dLTP>#8pmFwe?dslz+J;Od80wA^XY)n}h* zLMiesHeZ)F@W&V7ygq$hvg;CPxdSgs(G(m&at)X`*ab@>*0@7(5Ve?KiS`nXfq{0Z zZa=nZP(8BgwCnZJo>X@u(a0at>sz@PQS6rRf4t8UWT{7Oaq2PuIF(Okn8!QB6rD6Ufa<8!_-@s1Ou>{!{H-vz*kYMxSU21oA2U@kwhB3z2H0b!V#J&Gv1aA`SUEAF`h zzVc+0t`<-B^mWkShPcBs5 zZfI&j3wSqIoU@C1ye#_7bXmwdUy%!*?>fAw*7#e+YZvKm%{SaXH1J(*DaPrp6Jld! zU6|OuC@6(>akYH}eEDVtc0vmf%JD8eR_0nYGZ2deqCf4umM|pm4dFpld>sqIH}?KL z{rf{-iX!%sJ)cRVrNjF|;uc^3!4kmC7u7o@jp*t>Jn8$|N#)WRt&`RsAfKk;H7` zNo~FZiBVr#1^@voH7sj6|5)2PGazg6_bvwi*6_G?dZ!UjtzA=ea@vB*K}4F8{IolK z_EUjQx&5$A6Ezf*WJ$5R<~$j3;InkXo#Q@2V6YJ@jnlYk)`~ z#tM5mAh&Oqw!Hmv1`*n{Xr^K)6?joDa^zGe0x1owdCj`|5LfpyJI){RS)CVHV=m;G zz^~!|nFw?X|DM+kW{>@vs0!ydb4UpjAaIs7%4F!>VG!)+FuG(EWoAVj?DAF);G2i4|W)L_hNkT0f=%d@7vjYu;X> zI)uNw*&P|Ld!m5S#a=xY*3noNA8EH)kG>rP!k) z{3O~i@gu^wQc&dBcflBs z#{}={UF`>9cpsFO>w{n`CgRAp#FGA~A?sjK)yL|u$V&cDNM#KW;?~bG-%0-gia)*| z#Rrh+bw<%@mWazM2qko;oAZ-Vg1L|X6b27)&I5`bQg2K?V}7neBtpw2tT__0-hO54 zL^Y_njq`F&bn|4xiXzJSoNq!$=T!{?mcW>I-j;Y5mB~_tP*hsyH;g(Z5;b%iW*bdBB_!f(|OgPboS1e)Rb6ZjKPVCKYI^OIuYJY_1$B@6KR(6rAjP?eN|G^IbF?0f~Nv{dDfEHE`Ar<17N2TBeF>mkeq&RUTu|d41QC7J!j0sG8VR_vEd+g>H{lD#bCak*?ep)h$#789t33e z|JYr;2u!8qfbNt%>>~$~i?re@)H=(Bm28La?A94ie)e*Y$LfyoNDM$HXD3}Gxq1^F zM8$3zi0Nu#O&QGJX1@H4=+V^>oPsV2WNfSz6(ntWNm%>HF1I;+VqrP6^Wn}>w ztUD>lP7dja8Z{4Q-lt+#x`xs14pCa0R*34dGmH;YoXb-CQ=+69fU>XOenRyh*N}_2 zCQqsEC?r05-men>#uYWuGp8eZWVW zJ*OnUDffisNhMnC7HhF1w{icP5!XE*HS4DEr9+Pc-H4^bga&07)!fZPp(k>XDay8l zG1i&-13udGr`;nA{Ef}-2Z!-l8-8kS5c&``)gm9Ui9hEY04@S1T}AOvRrRlU^ajsU zWhYxgdct@8^-crVZkEPsrBwca?(*s(o39jSf(}ml9ELd0itmg!2k8(5+h=(N;u~rI)l1(c;zTLYvLCp|E z^5W*)m44HJ$5}^1jx#Gcv}fnc*Bfiz4frSpE}v=ijqpfzEg|U^gt(;1i0wLm6C*Ig zH~`nd)okiMzd7kY#3`K!Gyjuzsdi_#?sL4WZ`r%ZH0~CA74_^=B!B}yaBWjqsl+bZ zI~HX@F&3nsB(S3gU3*vzSPIkEn<|%Wz6M}Zd`k4)P3stO>8?IbGbXQC)u)ARAcRM3 zUPB=!;<@IP{0%2MyIpapf(4XWSJolYkXwR5XB|JTuR>MyB+Y+0&nh$D`y(WlIDaJE zLvo{lPkVWDe8hLXtHi%Z>b_5E$58)~v5TQe?5MTau3B0Xq_Rmv{CxkzeO077veNj{ zh@h;Y?E>6&r7hPz{wQW6Cllf8w$8CF_6ky1aoJ9$|4&ttc3%Y0-ND@TKR^Ah(Y_5( zQ#m6!^J~tHD2Y7wnpUHT=vZDe1%)<;VhRRWW?h8@ zPdJt@rA>CDR`#4cZcP}m@YEJ8&uuQCoU@nh>O|D0i~E9YGBXyk0ejuJE1@-oqYa7Y_%4&l z+O`LNu^V@s2Jf@Y#l4%)J6=7n5x22;p-MZ+U05?5?rUw9FVdx(%qdTUx4?NzFL~h$ z>@^p0ykpYHGv*NfMUm(MRh;HyWxmB#K`Vpg;?m)e4}XwqfUZ=Nn$ne5YcA0JxrTp0 z@ExYv>?hbjH!5~?jn8kUfc;8Iy{a#r$9_B>>6VW789&L9iCU9x#PRNa@=K$(DXsba zd%RBRg9k|*hXAsi>C{(uhr^;x4!eht-d$;%MaDSbc=DTYA{!jPeY>!GRtxQ?8i-Y= zj=AM%%f4?xu)N*<2BSrhWGmKlk&uv3VL0tO=HZ4`v99=yq>`&HUEc&wPqN#(njFOq z@h(xPS#&xJ7=#e~W%ke>E`YmR`+E=3{wLy)+QfkifkSv_Cf#)sVOK;U$MSQ%Uw|w4 zY_zN&!1OpK=3RTvXd(y|MTI6HWg6bGb4AbMBsP z*^2!&-e2JJzX$)vf&M4sU*_^ZQT$I7KM~?LzWmQB{$~~cvx@)3jsH2wFSq!gDE|LN z5y1a)LVfTr+W8OH)W7?+jUEy+VgfPZnGP2Yo*)YXf-3EXD$3yPzF z4mu?VdwRl?3^h~UcI!WnJp6J4D2R)k)aLe|8<{^r0!)ClQM;Jx&BA5UWQg=1G1}j` zJ=TvFSO-P(?q08%VwIPde`DAq`t$eS%?CVmL3OJlQeXV163r|2OVS>JBB~dDl`7v`@0uUofzg zI-GPVx9ErpkZnF+cVdy0lw?Tzs}=?%G*iIS)~Zc^U9$NzQ&t6yKWy%80v4XyP*efq<8g-4bY6v~x5YfipK^QK(;~ywP|#oj5f}j$5psya z9~>iJeH_fmBF}f{Rg}Na`eibQEHYfVqyYD6;%j_z`T6;vME`Ny+>TJ*rmL?tuz~H3 z@Z?*vO(#3>7h}lufVDOI<$-LKmRsL<@N6jGzJ2@pIl+}q)?FXBCr=_jwTj6--M#}u z6tJl!4p|A%X45$gn>qruWHP2cc-TTS3VAyJlC${j2rChBg4_AN^OW1j7dQlXdFDI> zb!yTmF$ME)LEl+5L5cmx*wBAyk&*E*vV)x21MUs4Rux^mEsFf6z!mX*V^7z#C)$Mz z1HH2K{A7Ri-DRhbiMH0&>Kt``r|ytN_Y&hqi(bjY7@$3jP@#NV)0Y6`IXCB9yJ{)a z*{<`LWxO688#$oh;QP=gv#lLKA2aSG$Z~dkr+w0XDL1c;jFm#30zqO+Hn4l}85D2m z93pYDl!?BX3%m|6GBZM|?tbgKduXa%tX0zLud~Acgn%~$pI-PsnoC>eE_pq(t~bNY zxw-f1i_j<*$Y!>h*I;;gLJ@;BSO~ha<6OHNF7A-zSJ%ClSbatAD|EuKSowhGAg~uH6*jBZPpS-}vm>kG0a1cD*tST`)8k3lqr2+QrFJ3TNcngsx zreEXuG9)CjvQnUs?{oG>mDkrs>VbzDZDs;3QJC= z@!D)WwD{Ocq-E8VD{1vCxXDs%AAaha`dk_&dYa5+Q7_*>@MQ!-K5qJB^%KpmCE))L-I7>(f9-rk7h3HH$ zQsIqE5oR7kH}VH5Y**h5W}NpdI?do~&?X=ua) zUDe=}&AUC~&SA%?P8Ak`B*T8&&*vPbY2pl4eCC2>iDN7M{XS~ZJ!c>-0y#DA#{uA2 zX}jw-=VPE=Pp>??XYEm1m3N4MK#qZ~L$^8dwfGC77{;+O9AN>pRx3-r8x7j`8}^<1tT&5xl!jO6&rJvxyC{Q} zA6#|myyo1eqpPAZxP0e!dKH%I`C+^XG15<8I|%4|q_WX;>bN|ZtD$y()EI?4Y=#%= zRotfH@SXNr>;P=?a-7d9&NP)N&`NH?#!C!z7OQJ)yC+dk?n<#Y)Oc7ezSf&2j_#_zXMroGkS18Un*VUHAgLc1Uqi;dTF&!##-pMvWk>ac@xl5{GI=l-UXW+xy^XSgO#sJ#6eZ&OG zE!1EX{XWMBM7R5tHookSICayDUh(^EtQIx zPL8|9L^k4VLHN6v;Bi2MRY6Y9f_|5b`DcifCodedrO)@YkGI?C9U_8uTk<2M@qHKq zhMjiJ_?3lkEK1=o4!$Y0p*-5C-yF|Arb@5z-d}OA6&C&;9c`%5Q@k98iG-i|R`h2o zz@J}1-kRGL{dJe|NALqKK;Of}ct;RAm&XKQfg|j_*qVwnBsPbW+zHAr-vUv0z1{rW zo6xg1Rxu9`qKm;oox~Sh6=HHZZIpw-pee`i^8=`B?whU5bXnQ_PEP+bS1>L12Qxb3~!IY{G;Be6E(pr7(hY4F&e z-`Ua2?UXft5nA$LDub82~m?)=`TPf4U~I$8Lv zdy-jG5L*Vz{efZ1m)r68ZT2m!rfrKT%&pEDt_Og3G0n_r zyX$Wf>h^yl$Ci6}p=7al0oXe$2$fgaCa9AU=l1JC0`638|Dxgcn2bp69gPcIft(eq z?Iy*qZ&q(r886E3KKiV~P1a7p_t{s^D`}UiRty|k*Vhgef~8BLyv?PO)BPkO zW@evZ38Vc5O6R0YrJ$*LH@lj;TcIlT{VwJJ6r`(RsY{=`j#(r8o=)!qt@nt$lb`-8 z{JqCYeox}Dk(qRbJFa)LwyOWm-ZlQa2NLUq%R;VinPDVS1T~JalSI@OZ7(mEudI)J znk^#aViID#89{?777(CbQZm?D!_57lv_%@fiN&&VZrnQ6D@QNB8E7NQ=Q#P!%BbK9 zt?$-*KJ!*WFVy*=1HA9<$wl+j3(8a~sX!vJv&1B$pd+5rb0?nkZSDIcrzX9MYg=MK zn2W<5&5p7W^LID_%B0_PhdV3sF2sxULJ zcmu~q`Ghfe)Qf-7b_-}QG4_L|Z} zdn~mVIFxi7oLgCwB!mP~ft`}P(Wtv_RGOyo>LTATy;h5A)`)9FAW!i=9iOb0BgqabR)rtljxPBI!XzE#EHjOyoJyawa5cn50vDuM}v?1y9#g~7iTws1oMqwayMbGn&dD!HC zsZ>Ex0RZ3ogq5Y^83H+s+8-3(Y}Y+Ru=%m64a+iPj;nnQYrhOdc#e zS*AJ^+nM4?aU0evq9WWvppYXQ!+bz%RFsWV{oO9HOTPUX@)@*OyVZO)YIBo4roF0D zGWClm)(<>hk7LnEY<+z?!@nuxo|goJ&XvP?D#Q(WA=3?BT9+-<+=hD#+dixFWBTtG zn2h;1FV4?*)FAxr#PtiV)!6N@*!rDzrr*hlaBzNO2(0TmUKp*oP}Xi`h4h1$uU^HT zA58F$a?pIgWfN?6L13p)r#6=J8+zC1fu_ewd}(bLr_G=}tm#k1_FQPDKZ;F6XEp&c z0v};jMt~tapW50~3=IvBoWLQt&)7++XS`B2DnE3sYZ>3As+^AQ6%$|Deu2}Lz>&jy zW}iyq1DRcYDclg}hxviyrtLshy7p3!*G`8OVuO}^FT?J+0n54oq2()|0z27mA3nt? zAlE>gD!jtvTm{UC$Xz7MTo~c&lDRnLy;>AiWvy4HUIMLOCoSX0 zpfGS95X9V&vU&WmP-vo8uSR`1DX%Y7we6wO{kWKg1?AiNiOfmTfzkYYS5rPz;>Tsu3eQvHi_*?%SCs8%?Xl0dfuXoljQgZay&|mtLF6_mfmh{Hl3;qIgK$IbnA{3 z8_7eST(in~A~m!1k)`7dI+2`37O77X(KbsGwq&{TP_X=7+979217kr~5Rn+uGb!rz z1}T8&bM=nY*=AF?ZMJ!2|*P9%xF$rmn#{7c{$#^|V{SX35X=fD=3wYFsaV*VwOTjzE zI^wJBPrGmzBZKH4dik5L@94a<{P+P2Nz$jhcRIwysbytls;Huxvp?ZL&0Wo$9`|U5#QX zfASvGAuJuL?Ch>we%z!p3(YZlnhPVFp&Wh_H$ep;4H`GcE*Cqn3v52k zUhq>WaS=;dXv}pAoyOVo6Vye~CJ3=HGan&%2fH>iQ#kSpkc9&XA9st**R*=#a`8fa zhyD&K9=lGDCf_|lE)FLnLiNqIi;lgvv-Rj?5b*uQsNrmrW`924qsOS;TY{T5dE5}B zYh|@IJy7N`fdWM_IXa-Be*fW{v4vQ`!Dtmsn_zjfA6KoFb)Mc>_xU#OyeD}m(Eh5& z=-g-%2t6b=XP0(8$S#rv+?oJnkL&S#JPw7p%YtexU~8bA4jUgRP{q-6k}@bra=kxW zuL}79LGGs2joCBxv4Y5`2{nyJk4|!u;(d9aPCV|^@{oDsD|&iY@8pa>s@cyalVtRA z8iTpIq%3jM36+v?ir{+6cs){nf^R+)hyFD^YPWECj+3yHZ1HjDGydNAbDq=ay{oq* zzD`<)OHAAl{n-mZ{J#J3&TN2heOUb+;!24j^FG{Ki}#FAUWjx~l)tvJN(G9b)$h^t zv)t2H#t+n#sD&N(JOW0ITnb^d<|%j<9JM65-9Q#13*0IUT{tH% zZw-8kO;IY2AZFIEWei6ZA+HDLp6-O0xvi83nq1ymu&aFa87 zMxo@X+u{Q#fcv#C=cY|>Mcp~vzar@8NX2emBPXwRoeKBy_U9|`G#*rR7|pG6R`D#| z^#<>`eucR_wzM6OM}qE=>b%)1`i4M8T7`J}%E;W~tObK426ftus*pI_?Ug{Gi};lfp^o+)}#9S8*;t+JrMG zY4coj(76J$lZ01-cU8(uI$ffk&l@-<%bJG5fq5T@UZO_q<$b#Ckc&;|i|l(^epD8e zoi!>I<=Z#3>>S|bL}MHHWNjeoib+h#M!iD$$hp18%WE9btZiNXw+|$aEhbz(crgqe z(yZG0Keg(vnB=RH!bq; z;n-a=FoQGJ-X`>-*o2yvdidd4y8}D7Y&to<{mH_rH$?o4z{8J=yN`h5)B8Y6?Q(&4 zTOSCxcg4-EaMW|pv*s5~J59o@scsNa| z#Sn16@PiSL3UW6_LgHb}nbYvTmX58Yn~#A&hUBWreBNFS3R)&>`K_7W0pMqn64T1Z z2V;;$-92!rJ7!~Pj|}aTl1T4Y*#?fa z*#{KsM|g3S7|U-n!|~mf8_lI**KP)816etaxZ!4kyR#r6W*@o0=+{{dsXR$tg%9de zKKaeE^<~7A>sx)q2V_>yGQ%^7SZpP!ut7_1V%qNk&a%n-!QsxM)qCE;Q=ZY53<0T? z?K6m!xFVIC0>IODt1@LrXdRRXh~KRu-BVEU{6yF`T%=(5uupHQq`{q6G%OXWErC?e zqf$L*=m+XOoNBjUP|MoQz)8#cvJ4#$w)ILkuQFn0E5EQ>cXg=XU_fdSFng^ zD(cj8$tr=xyFiY<;owmwTk9Ab%==m18QV0sioYuD-!N%@=Z)ZjOa8;gJzt`kLBt#V zS=mc|3;M!DF@q#ZPg%~dWuiZ$a=irLM)z7nBwQwqRvC_k-qX9~-@ru%FWfDf&2sJo&6hZ{ z8Zs(3-n_1<;#^)GJA_xTr7+ap(KX09D2CKtgB{L=F`cg;CXzHHMK4JEM)*34?L5-j z?paIhE^T;G&Mg1H(c*4!IDWN?SfEL6E?}+ds`XjTXEjy>c`?HFqozItMvUpKDUw(BZvDlqM zgdZZytV&iz`W=BFLwebV{ZswbOI-6cJ15V?ZakQvu+IxN-5)gfNC@v~%~=>AA=*O> zf4*WeFxS&icR1p`Qixw?FXN(RUUAjtwW2__E{}+bzjr;7i;8H1zLN@;mPqm}RdSns ztm(#4^ZW}-;YnVe_Q)EZ`W6&4^C#Gi@;C1(PoR0dXW}-!S&(PTmFGmcPIiUr!d-PTj_rIy#QPW(+{D=S1sok8AHD5NfJyO~>~&a{CGqP(-XdoUm;JkFJS z8KaA8;Qo74wW0zZ8JE$AK^;N#{-Z-G4!tOV37p9!s`)h!)v2aQHFjfw#5#NF4K#^? z+-(S7ZovND-BZ%c^lt|7+_9fEBzcmWPWAa9Co<*tQS88H2RO9N+ovx2TsKF$1ZvK@ zuUNuKLeC&77R3)3shePI2nn+sb|yCwC{WOvl16*;@UvA{!Z-TM<2R*;{8G5(t>uOg z{BE_kiN?S1P9Z-LuJp|cVp!ki=8xH})JAo3mhAC{yv2Ab5InlUFLa)=1EhERI3HJ{ zY{s{<;WR=lfY~N$3HMx>tv;ivuxt@Am{5rNvC0OJppD{TZgSVu=kw1cRoVEwn&Pxk ztBO9aBIt?Xt;`hp)So0nVE%Ky+JIY}8*te?3ljD$(_BBd&&hrqQuI`P+sB8A$-L8}v&2;o~t8r6fqxCvD zUQ8S5b{syK*c|NzXEGgEVIUk`9krd_raMz5hc?L^7VG(om||q}M$UpZ;<~co7VFk& z9sK;nk@y<&?D0Tb+6njlc_pzo+5i=tNsz?j9vwfpAXvKe!Qb=@sF`qX*j+DxeR!^U6vGvHHLAP+{78gT#wspz7D8ic?KWbN0bIQk4O6~Qv zq)kzoS_-;2PJ9Uym(K*-GYi*k&UIWD+?+K)v66(-a>G@b%|*n=rJK%$j^_)e2Aw8! zS|1MktdAEoPkr+~7L(tMkB~6rEva$`(gG)A_q;hr;$Bx>7V+v4J{DJ{f+^y*I~g#_ zg2sUsD8_jTiTmLQ@w8?xF$8(Mxc+k6BgV#B&W>gM_3FuwZDK-@#b=XF*CXQ@8DS6Mx%Sqa`4~okI`m%A4@vm)h)UVqzZ9@cEbXT0|9o`)>hbaUo3{? zjz-pSe7-L}P*fU0ttr}0 zR67)D(XKZD(K`-hJQc+dkoY1tF0OjX+D1+EO@n(@n)I8Yc$Xg={x(F=&F;iT55g6e z2>P2kNWUXjubHoYb4bY`xC>j#(9-&hmbS%`jKXN@p-T!nM>X3R&4c5}^DcQG8kfRu z9gCXy0C(4AoAVb+)(2t>3FcH*K@BCAFUC7it<+A>3>{0!-Hv69;wrM6Csx)X>#}ss z^NKb{9{61}lK4j6hfYw3S;CX4ThSHbU!9khCTCDiKxk+G;+u_2Mq?2#Y5#k)L7wyI zXXD@$#=|Pl*lJJM3g-D091#144jp9Tj_=q2GgCS31ji%$|U1o?; z)qTg#uEWg;ku@9mt{tCr?J^aIp!UmZt4EtBcBXgU^50<2b&b8NK*Q$OrApfKAczI% z!dY6wYMDS*YCGjK0(7Mll;SZy(U>fdx2)Woc|kkx2H!2gq5q)2Ay##iEb6jitnXxy zi)GiH71{NEc(^JN{Tq;r;OVmf6-}~fZZ#^;bS4FZsLG1GvYz1=^LBf4!$k(vRwEm$ ziW`ryY?kOBeN(QJlhw|BR#59vYRUMhC(r9k)v`X5?LDivO#A!s`bOnuzzbk95N!85 zHX@cvnvwdB{7f$ahSk(Rc3nW<6*!tP6f}X7>$ri&3LiZ2T!`V-cS_@=vV3M22@A>E2-$2rHJ+LZaHzLk1F5euTx;~+zhpG55O3Mf@qcJK=ioY{u8SvW z>@>C-+l|%Owr#6vY&SL=+qP}nY;3?ryuh$O@~srDH`VVq&VXRgG^9 zn8;eGCPHwuwP-!gi_ol>Htc(4(CchBQ0;hc?9}T>uLE-LD(I_!1>Jl4A{vG_Is+Q4 zCcDNb`&L$>iASU~xTbJ->ti0<#XO>kX(S#ols0FXgxg~5u@gg=`~~*98XudLTQgkx zlEi~wSYf?Xxc4l7R?;7t1mmqRQ_Gs>Q>0J)Fqn_`I$a-O#HWb&`Md@>$ zPdU4el9@vpzE>n>r)RDf(=pfQ4VtRaf!nTFTqmjh-6_lwf}srU%a_1AF4A@Bq75>* zu6SSrNS>>0ImDl3ZrL9-hkPh_+HRcD>$FQNz64TK=j`wbZ`hR^F)oLGrg@HMaxax9 z7;F^G0fN$}W+U5CPAobBuj4FJ zXql5MJH?1ksc5_73~VNWFQrm^tV#8A65OAxz9Lc1@0RD{v%_9Gk8QT6fV97{v3g!s zg}A9v9=uw^=CIqpnuGLCB>3n;9*%ofuB5AFo*8u{Bd?{T#Hi%uQ>`VMU-Eboc~ta> zPrE0}BNLoD!_S8Q;Pp_82<~~rpvJl?Sjqp#)MlMH`LkMDHsd|z?KBv7f8A{}5w|vC zzrBf{nMH3I%CtB;x?k@u-wQ;|DUv1loVbYZ-p6sEY3$(f(wijYjozm}XjtY8ULM7w&uYkVw2)I5e?uR^w# z-NGRR>ITm|_*fL6R5X{oi(1ba$KM=`iNV8Mi+^F5I^~0q6PM39oh^u{w_K34hwt!w zOhV#+lb1%V?}5B--)?FX3WRvskC!u!R~+~6#pl^_Zwc9LT6~a3!*eq2jKBVfPu(-n z`AP)(?=wsd<}ztx$11Se{;R}VtCQq3mstCdQ0e{_3!zn)H16yBt;c+Teo!%VijWsE zxU%U!`u)eYiudbzQ@3Zx(#@o!JO7nou4bdP#>qliGQ5X*D7>N*h5JLL!k^%}mTtbt zXz94)59vL9E|}jSFyH(If?&}jMg8?kpIu>LzJ>d8pw|D0B1e4B@Ob2{wI9d*8r47; zTx0i1koqyKpL)Mf>Nxn0G)C1?nVi5=k|hdVCZP0-fpvQKkh|a-T}^$>()07n%~+cq z91q5^@g&^T$jRsn3qjn%-lxnvbZZdQcc<U?d?bD#L~3WHgRj*S~q({sgkgRj?<5mQN;)6=jYSCduoxr05@5*Rzq*b$d8LBX8OR zXDY8h!VS!+wV2KkWE^R_9EqpU(lPr(nlT0X zw4pfjG|SSmPZ^T+-6kOG_a?>!&Bbh1=(LQEF2t|*vIB4r&KWPdcU9+a=__kk2G*A!}Cq!lK=X< zYNG_$H-UejYx3V!>Ix;RFWMUHUbT&7WbypTs*O8Ul!>|)#HeO_Nf69pv8(d(pm zX}2HW3`-NrHMt9k3BNZ3r(;tUk3w5(c`D1260Vajo0S~^1Cp4nyS909TOwT4e*{Dx zxY}wkOk+id7~ek~wWxD1^0Wmgpb#a94PZ^a+62Zn9u1Yh;5f=&y75|zleJ=i2x(g;yExw^@qLMtBvB7pn?0S-$08Dx$JyfBEq6yGG=-| zH|_5VyCxpm!#+|tG?I2|;1Pd;KC8XbE51w$j?I@(^3JDSIFxXZ+=J_OSAbRxAs4Y` z0EVTOR--5ohX)#WeEx8Nrj`7^l`A2V@FP)J)Ji) z`vVK@82lE45hGLu`9+cfxW%*Kq&H8P9j^-Gz#~`_*CFTK9A*2kQ-kFosCOWGJ?#h- zk8QVW1ibwG68+;bvYus2gaTF_n1`fU)@6Sf^f$^ zj;%4zjjE)(D!zyldsmySZ&eN4GF<$hX{;x!3#+0p6ofdpR<6y{bwM5tbLX}rO%)n+ z1q=PvTX6+@pIXKPOD`cO+YXJ*Gi$xzqkV?8(Y4Uriv>nQ(Tl=A-WcP1yO-T2F8t3H z$|&qp@#L6Vl!db!td4;{k$J55z@@{LZ{c1RSR*x;b)@#^2NzGRuY4d7@&fjj(7u-nFdDtUUFJl3j zDe`9-)C960dLD-LLB=2RXqKSkcXLpsZ-JFgL-Ah~SafP*x-^w6(DHy<FK)@pyb+0k|;Ox8F;=`J2D5&Y-p^WtMT>NHZ=_(o`PIH1piRc^4Eo+b5IEM4R}n9AlbmRjZSiqYZi;humvwGJh;5_yHu`LZ&V ztSFah`f&L{+Z z57z77o~1l*nV%nbNDY5xLkB{4ONB#m{*?D%Rw5W{tkR_NoOe3dq|^GQf{lvCw-eZ1 zB){UQ_=BqCWyfIL=KJ?TFUi!~>YK_W8aV<9VJzO5A-_fga2J6f?H8^0vG^ptGWWmi z!TR1jmBjHR^zdrZkQhDJDV;Tpc8c5W$GZ{6LwekRPDc<`-0A)PGnLoVwcDM9WbQXe zgtu04%^)=^L3<4P{nKL*zxDi|Sp(D8%DpdL|1P7*vxOsBiN%85^u9WrL^sv4{Y*o2`Zh|dRRo~E?WfjFP{(_X}9->*XC<%)}iM1X0 zY&|~K3WIiY-SkC{&|NKr1nE@tjERBfsTn^#`K`Qpup@FzDitU#A<<3@kb}CC0iFP= zk`C*LSelgQwq*h$<>lJ(yhnkDco(g`Cg6DAT1z-;7swv3<* z?B0}L{;FJ0LL%Z}Suf6UXtx3N7!~r_rc%&vA@}Vj3BhdMFWb2Xz`;}Rk{TO=iLYL? zx~{P&NUQ2$KOEqfGF7t}Q?R)^5W%v(a&p?|RX;+ORoPC3z|iIH!79OO`!3nKPI|>b zn8+*;YUQ@^w2r3i>{82oe&zRt$<62ImS;ijfuPRP)#2S1#Acv$TvAu{{N=AsDy{U* zr`5BAbc$fh+uzNi?}Po+WYi14oe0^R@U5F76=4H#sW!mPs>jhU+i5r**9fU{Ddj7l zsYE$S&8?0G5n$9HeP>tfkv0R`DF=xcxyya6!9+kQZ@QgACD;MsK^d~UKTif*zvm4i z?!&WnzFdJN&!ZhHy*ybSYgGLDh6|M`1)M_PaF#r^#Vl7xn)!sM=lG?6AA}qXKpacP z5cA(#=pQ_bOJThor}PqWBK*;^N6Zmig~F4C zj{RRlZ!ihSr6oqsUZ?q?H{$9y2biR^Cs8FMp$hm7ETeRNb~{iZ-3=_UYA$${D&`$+x)0O+PQgs%!QqQ>9k<@BLYw zVA2EkJWB2@YM*1s4ElO9wj1t59qlo-cgh4d9P@q?T~g*g9WnBOVeqdefw9iZIdCuEz0NN zvzSJ9%MZw0MRWY{NDTlDOCJ(Z(sEbx1W4E~gtX1+;G4TIkCzN;x*i28FudwpRjyF* zctkPqNRPk5I9CAHu_gE`gk1Zlehr^8_-px9bqKr`fk& zd;T);G~FZ{woi_(dIWUgADy97?0m#M;NJkkKxb@^XR}D?xut?BkbU3v9H8gy9CKke zzEpYV`act4(ViX#`NP5j&<1?P*X6-Cu0&9hYDQv2@aShoP=oP=2aB=Ij)Kd}vVC~A z8DQ>`bq|K>R82tGvx=mVG=-Iqc8=7fIE`XW9-b;(jxHak@?ZTxbjA_Yf|kQ3pdvLm zFpsCUSzWD-Ipg7hUj1Q)08IwEAw#(wosRpV-=f}C1Yn<$Zo%v}#9;njtx+gLp)gnD zqjQowL`ehk{Wcr}&UH~IRhd33wMO1OjBKz`5}BUXN*+yDD1al4O{Jl5&qRN^TX2#2 zdbpm7J*NxssthIDViv34vitQS>b`aSQdBMm+4$mI?G*?TCYr-*6aV)Ro2ZFcH1`}U zHUy@S{R?C(dc?3E_m=l(>G~nw5`>f4A2xUWH1z0o&Bf8Y7 zKSgmu^zp8*ZWr|rs4>#<5S^6_>PE}`5hhy&A8@t$~2Z_=AWXL14b0*kjR3YIg zoI-uKv5ej)Q|CU$kKli{%rZy)3IctLayrmqQOR(rs-4fF)y$Ev5lCUS;r>n`S>oefFne=- z&M{1l6b@nzCHj3Xn4c&{g@(at-cXlkKZb!k1gNfKgvAiYM(w&B&TT_#9o>MkaTXC> z)B22XlZR_y4+j?iJMDnX#l6*Hr1CgIh1Ht(rA`OOsJ^sCAX$H2ob0v&QQC76rk=at zk~9%Qme4z6DD*ke%zMzoxe4xH0-v{QUlDge{^tcArK29$=MH@dUe)($+ZnYlO1q=BF1{&IH$T2%n}UsOf;pW5)h1JT8~5-1=+OziHVK}|XwBd~n7MzaS) zw>TkgkWFW59EJL6$?_D{i-gXX9iRta8GtXrJwnMM*nvPHQ0@plUk$*7!E)q82c4!L zY?Ae$VuNY-JO=+u`Rz9}m@7-=k4Pf87vDuI+&6nlgk{b@y73xve5TG?Rydnt9i9@| z9{i%xe=qGdKwhs3v;1QSYd0MGA{*X;vr_inS3h3ppxxqgb@jXI?P`;q9WQG;n!W*- z@;|r6^hULfxt!_XyRy1ba3(<94*kBB3k5N6Z-vpGDJm$iU@Ho6*i~? zX@#U2SOZOfXrBStxgttmqR(BYKs0PRU?wEf%QZVjlWfKV(%G6saE%V6jh6`)r~ksx zN2KVsxhhGAJ#f|BTX~kuE;zLnuD7ckua1}2I#^2M`|owlxNqus%( zRWpE`aXH^2*G~wD*#EseE7E5X{oUnikB+` zu*y;9yJGhlw=+?^Bp7j|uv}lGOW&&q@Q6;by-IlkqGO=(e`^dV#V_y)L2RFczH^&A z*H0a6PwK>J4p+KI{frno$-czVUf5W%9{gB};iH}-^kOvPUqx_df4!SbS5^;sk=)7g zs9%V0{MBBWJN@MYcuNbr^);5MMzvS=!D6{0QeL`N*DsDdW4;}kAZGRIH(a8r=Nn$n zp`~-x^I#a-z_|s?8G83XC3ZwqD`Y!JpeXD^eC6 zFZcDjBpvQVVJ7Y6!eFvs?z+6TWO_Auqrw7uEoKMVv(Krqe#`B;2G$z)ZNENeBTxn& zwX~mqpiLV0ZsM=8tOyz3mlN=>H?HL8M5ts{T-0PisVTFU@%8s1EU@;;;9NvY?}k`; zDXoczB))rIp0H)bUM_ecVKA=uVyEa9PRC$+sJ9S=i?*$<9O_W(u713mKJHB3EI0A0 z6Yfo;pol7405g%sOk^K2E78tlW?m7K%g{i^%qaM>XvaALMbvp2H!< z^$l=orK)8~e+wulfZ=Pr!j<^k=GWH$xFe`bli-K!>}mh#XQS zN2f#z;)yj&cK~SP0dU5{^s5Dxi3bY%vh$OVAotnr^ad>oOV!uC(h_QPKHuP28Z&XnI-k~4(dIrv zI;!Hkknp2RY-?4*kJ!+DmaC^QOQElGZT$;K;gwsm1K$H!kB$(@&Qhu&GP5hRY=;=f<*i3!hPZ~E9Fs>axP<@ z0yuQ^HQdgn)(OQt{i{iSnH=Bqw|dRx2l9hfIHrn==Dp}@=QY_}p&;uO&@wSAzwc(PbjIcOHF43m zS9!LZg(xwD>+1oY=k~AcZL@w$uI&ljJp<@1bqMOStNrI`^$Qcyx)afv2X99o$v^+Js)SVZx?IeD^8wt|wc(V*dc?IBidmDS-aPe59FZ zarqdCqtWXOWv4(oCX5XRS7*-LyaX*NEEgQ3MDpk6u#H2A@J?cQxlhd0S`?d4eLelm zFSvo!EI;*$s4X~KL;x7#LDoDdraf*o8JrYad54+%MEAKk%y+CN(R0W;ve@r&}ggw!;04p+s{XWMpN z@2XZi5$rYc?|`xnO*<;%&%WNOaV;c zi^G-->IXxyyM7GMjd*wI8(^Dajy?v!QuL+)T_ckiY@@-*nxwn0G{dXDq_+VugyNqc zFJTa3Gu1l%@3ku9u>Avm^oO~qGRC&b> z@lhU{s)4C#d$*`y>+egEnk-KnU_1Q%XFA`@-F;lYbk7-_WR7LJKP#F2SbxYWw`QJ^ z5h^`&yl&h{A{-+E%4*cH>uSXRE%#k$Yk6Xihl^*Lygurp3^HE6cj=0GVXu3_l7jO0 zvJ$K@IgVmnrXtmc{=?a`GmD-)l&upzFS4<&be!0%G@2&oSMyqB+XZUUD&w7A#adH@ z5{E?%=2j&!jmNRKNFHnbmd~*sq~QbNT$G5?pmx4n@7QxIRjAQP508A>R92>H{1-jT z3+0yG_ZZ=Lu(d>5+pe=>Gc-!t<9T@RSL61Z!~1ak$(e5vTT7lY*7gf*h17m`yA5Z< zgLfi+=ixGPvGH~tGN;8Yy&2D}y2$gpLnP?zr|(&6mTTIdb1|!Te-p4Sttb#KdPcw9|0a$cw5n5eVMIW|1CWDF<1hQ&}()L zrUM`(H+Biy;{cz&pfmxReu6P-^Am?gn(#9&@g_1NI~yC@H2k4Q*YhE6%Ow!Gz+Dgu zGBYvLU$!9-BJo6+UvlMrafuapIz zb8m!)GfYq!nPgc@-xEUhdkEn@pmez|1Lax$$GnSDgsisPm)kkJuTXLcg&=mo9Bc^z z|Kjw1z$Mx{442#D{*2! z_pjY`)B}P!`LhpgJ8dg*T>>a#vQHNuUp!47gQE<>JmA*O0@0%#itI-m9UtEN@*T&L zm>TsflNR=-Dp+Z4YUe`KI~pJAJP8QoGjaNIhrkg8%sYD|2`-#ucFooubqJ^Poo?ZC z%92xAX_v)lwx}jNv?5LFIoz-G{RZx?cKRwNgpJ(#_hPih0!>c|PK9slG_kv7aBG1= zH@8pb#m3xi%^0ZB8?C22>@D!^sH%9tA_0$$EayQ&aqDV{klXUNGM+EXg9*|e9y*if zw)ciO6@OJ+taU*d&(pX?%i_(4dT&NVKq`f_+N|NDhJ)xgt{gmev-=^_+yafD{-4t2V*l&&Vu77`KbzI8tZ1b}K>}Al@LQ*n zY2k`XB@aZ6I{hSCwFnJO&X*?X3(q3ocLiA!c=FLkb`WBY3VrepuxUTbRQ2@_+< zqWPq`o6nf$a{*x$i8DPTOIy@S-h48Z=`RiQY7v8)BECK}tiC}`>9_t1eVt8(MIq;A zZJFRY!h-aLoM0SC@;xt;c5`?dL_d7fid@6#lzm6~1k7<8Iq~}!9<;u0=+W*GGV1#h zjUNGt2MjExw zh-Xu*cI4hH2c`=>&Eo5)=vQl!FSul7{4i_=Pxc|PnedU0MybU0CvwfO+r%z9{%0^$ zP^4(BJ!#occA+-OnsKrmoMR(jttWx4zdf93T+`-HDN9N>2Xa3)z;<>nz0#S#rhWQ4 z>sN?kzAV`H*pDhVzkeEkC2x*OnI^wP`1R)+*g)^O8znZ@(NKK2s zhkf_?9Ak`&qFyw))h+_bYCmr%|g{>(fpUaJX#GZjrZf z1k)g`jzZqL+BUWkjU}5RQGB%y*%~7E!~MBM1c198*V{S7ND&y6pMt_0->x7N{yHu! zsA=`#>Aywcav-ipFr_NByI*%xU^QLkC=S#7DHn|sS?|S7q*UmA%q72lA9@Th2>=w} zm28j%<)w0TzXR$E1L#W1(wp4{EH8iP;-dU*+ztc*BbMt$8 zJ5K7LiGhD6N&DtnOi`(YadAWu-CvpF-)#hTBJHG>d3LlIqWqS?lh$Z!Of->YDsHPt zG?;AprcC#wPbz&xo_uEM_sE%I`Q-#qeKcb1Eu3b!gO0$_uz#fh3;jkL~61+vnkaWjMJ4~Vw*`z)GKCR5!0f# zf-ITo!YyLfjQ4&geubrGy`igvWWCWmkr&Uoh9%uIOK=FJ9y#9ogvAd21VP3n+z1lB zc6?!&0jgYBo=Kj+M@?9_ZwH5TT1!N8;zW{QR0>J_mElYG)#jf~wT-Zezvz?v*3>HD z>9U;HG$N~FWBDr%2pQ?23NG~V@+o&*$~U_?3%p*{M5wx`l$o+NG?)OxT3rX4 z_^@X7lywtk8t*@Jh3y4xzBaYs>G1ZGa`4uzMM79Jo$$X(2VGT%+BonLTz-UIOgSER zn)dW+QvektOYqkPI##H)?hNbluT`!__;T&1J1>D?!LleIUP8p{g;$6%EnFhL+wLw& zhdn$TeZ~4MMkm}!R_j%JdX}l4oeDuO43-J9)<9f>BnfG(g(tGF2rK~ z&ua^IkU`+PGTnC>IG5gaBCk}`^r`7+_UhQ^9L?Bpjdi?mm_PlEs=dtapCYK-nAtVh zB`h=uVbr>-6CY_Vhhs^-+0Y+d8Efo3iMv6N>ASzFX*&c_jU_wxBf<_g-4ygt;&>b! zB=aT??uMne5%D;8zbh0A(4btGTKf;A4p1NFr8jr{YN1oiEny zI+!a;O@qr<<9dg;;$w-i0kaqU3AjITk-k1Mt4l2lfp_Qi7YdkL^Y~%gqNYxd@~`Pz zBo`Z8Y<6&$R$3EeOv?io*FTn1Qaiv2?vBkr2HfTF_P}-V$z~-rgrYrk=~4))@ya!& zGuajP+f_+oqN)>&lCr@imm-t}25KKf52)t8SshL-nfAo#P?W*j-bqm+eGx;#bAI1v z&`?||w^kwmLnSLY^(r}RiLKd~`Rw#+Gg-(WBnhr{8&lO_du$7J{?;4>J*|54J4K5w zA$5N^H=wXQUruhHt=?&go)qcd>Jf=25y<3n0iz#xaru4dj7q3Exo<(FK6r@lFtK`! zIqZL))`_~2ZGq=q1jQSBvO4gZB3q4Bfja(YVLmW$I&q<3piDda zn(bOd-+X1P!@;=Z@ERMVgWAHO< z#A;9s?KKqab=Nm`J(xFE2N|<`m-mlfVr_oVRHcD zIlO8CfzVDy{*SEC$mw6XDAk}~`u z(EBYpS{&e#2*2Cs{JNVF1DF%HROV5LLXq`ySUQykjp}y1!d@(WYg@2j_6pm14vXm4 zhD?=bTifNFg(wIrC7$n9dweqk8@T0KD5zuS93xgfGYNGA^}^f*QY+{>c_(M;w4OqJ zqKkd2ZEi)~E+!KImww=%D)w-m?7_f?yT0J9W<4X6KH0w)BJg;z0kH!jJzFG4cy@VD zmaJ=73h*4Z1WJ?Km+iEKXT3piUmW-^+izu;YPu(KCwf5BNDj{!UL2tiA0Vp6u-locyCgvmCLvs>lE zY85w4ylLw*r_WvZ=@MM{-znQT?2>pb5y3YHAIkM8JezD|yApH9{JQ>1Zs*#V6rBUF zwKgwRNJgs5U5zdBMtRU|@0Sv5)g=LmfIN8Z0@92^SyfuST&YQA&AivKd3|AO(sfGh z4`U9_Pz=rNXWnKW8+?@v8ySGg=3Jk0$l+(Cimm^KuryXh z*iIwkxk+huf5u=@aHrWzo+I`^_}hb2kJSWfR!X&k&VsDh4%y-q3KHp@yQzF>aijLs z?fH76X7>B!;?n9Q?>E!?C#i*UvVWZtK(fIV<9=N+qB{rMz8%HMR=sq3#~+1p#u#K zTKzo*(eUH!5AbW>zJGXe+7DY}C)1D>DO-4im(UavQh@8DAXPl+1j9(YVWfiNd{j)C z+ls`+wsb`00N$ylm!rk<;tewC*zn zF0L2B^Y0ReJY0l{)eW*+HF1kMmOeg_ra}A}xa=f7;zRqutKKK=sQK~7b_wfs+T^cd zFbdpIOfp1OCvA!w4*^;1(r=3_$aR^TZJ8tE^VNobnLjp-bU#3-^(UbTsAZB$Q#^$_ zWu+y-D9@U%VOD|W?lFsMkKrfKlx1M(xQi?MYFF>I{qp^nI{#5~(Z%QgX8~lU+NWPz za;)x2CEZX_)o}MUPDHHT#2qYmM3QDLH79>`B&Bk@RW_QloQ>~Qe!n9`9r45#RL?n#B;Cn~grIl7}n$l9rWY3;xKB%wCtdef&&zZa~DLpC>O) zsrDh{Go*X`qbcv%sqy7sZ;|$W+skUD7voW|xOv*p#mwe=1oQRuT$|fvv1gaG2U=)R zcLx$ijSQGnv$^f@q${;rB`rm!a~un&wa@-=>>!jK+OM}T3ggd@D~af7$nM6=e}rC; z4e4jB7y*pb+<7wu&Qa+4y&pBbUpv9!1R024v~c@bz*J zi;4`9RQH^rnl)oUnDj%&E5vNl-vGositSaKyc>VNAg>~AmwT1Td_b=*9*z_(rh^w( z7bE7j-GPziKCk~hyUvPY{;T5M&21z?|oc@{zEKi3QNrwy`L#a%*#3gCWF6NSUv#Vw;9S;jw9N?vzNjCcm~w zI(2d9ZLYy%Bn^FU2_(?%+#Sg8t7EWu)#mgJj=Wfx6)pB?DoS5Wyk2sN*WqRtOBmSS z8>s3+iH!yi1f8bKvVs#!br)iJeG^ESXu+7$srF@3?$%3R8i72BWuUwQPvU#t+qPYH zV-uVxhd_BsJ@~Fy)T_%jqdVh- z-D*Na#ycRVa>g@1Z1fhGqTM<4?aTB>#0JFl=_?IU{d;ADTsQlJ(cEIkAjDGOF8-du z=N|rruI0Q?u5q)3)A0Mj?(O*x6jF)0;h4$A@zOyUj*7`lpqu|f!`Aejjr%p(YCFc# z{cIr=>b2pW^ZV*end8d7M0BYLKf0aGO-I1~wPoVLPOZo%c3aWt4CW|MfH+rV5=NWv zG;%i@328PHg*bC~0YD*3*HN1w(hm}Rx9PQJM;jQD|jX&rND!kQqB5|HRtJBBI%s! zvb5|kT!lJXcV4CJVz~w>Qk!~*5lNTy$!zN>wuL?rgkr2c5D8{!1eo+HbHUQuDB1g^ z0}mfq$O+8;z3_h6J;IYkVoYdV^&4}pW`Qy=IT(p|O3b><_g&bKY`)S4ws={q7H)aHRyf#pn~DcKN17j@Ne>}Q z1Ik8^_n+r1AcUjrs{`x#$>=Huxy!9sE4#DBwWZn;=3Fg<-w*9Q@0p(ULW!%aOBHL~ zc&qvHlLl|}r)DlH!>f#Z8h~X7sP#&98gftGcAAe0?ERa|LRdaQ!IpMiC7dDtkp1&Rjv3XQvsS zvtzZ%GVo4@N4l6&4es`U>4WiljQAO~(LwI-gU0l04JRcqkai94DrRQ8v4@!Bnp5ZYa9 zqu{sy@_kOGbAcQG^naPDM%aCjsk=mH?S;1BUrTEFN`%nq9!v0EZFw6buGgX+mq5S^kk^ zRRX0<2@T}vjjFo%benZJLU7&!sYj><>W6o#zUmHz@)#(YC4oS$txR%8>>~2DI#nfj zxJntd%VeAUHfT3FfBQ|wwoZfxqz>U&@F*WOGJL>Yd;hBx6u$gCniDWqkI8x*pXFv}c7pUiz z@*pHbYSx7veC(Z|(o>Bj24Ce&qeEN(ZYdfRjGrNt2YRlbR=2-wNl9?O3!&;yZ@a>- zoc`%_h{>a#)}ISXl?jJTzAHh_^9Sy67bQn=%gS|>Tis&$%eS%ojo-wFt-%=y2^kLQ zEG*8cvDOIaj2AkbO1#+u-tYT+kzBdB2yMFftyb&0EN)qmm)MqO;X!DvIhF zoS50vs9XK5pp<-zZ%0sJ{0;+Cx|Ko|E) z9&xI_m6LgGmnSW#sSJfw)cv?eEB43Zk&W_aqCfXd1nsD>Od5NvRYJ%%VCjNtcQ?rW zddoAHpvKQJmHl~75=S9Nu%~@ajeg3m8m87}yVv81IUr4x`tm;zTsdn1h(F!%B&Tk0gl0Za|I=gIf-r#u%U8hfzG!elS_if zWFgwq;GZa0LXFZD!H%uxBBE@w&m%=93wI@zM(X!{9R70kj^8FZ)&P7csH&5kA?sJ3 zgTvN{)urxhANwH!SlQEL_~%9TbdG_**v8nHO)&{b>E)vO2TrSmyCZ|i3D3N+wI=*u zz*`-C_EaryP-iSQ?)r!dk{keaEhA#dS_?gFh^h3~>(J$G1*5NLdo~9fCA&*~QqNYl^jZitqH^@j$1f2>ga>sp6B<h`W9yKn!OcIgf(KHv=9gD(yv7 z>OgWK;}>)==9zbqv3sS6$IYn(GxhQ76L3+S8Q4fVe@OCNmyJkPp|R7(y#@N0-up^a zchmx7B)+ir!S`13tF~Mv$S0j3;sB}(Qv7k3trF2a|~5cA+)cd9DaY9 z;IO^~DI(ww$&YowlJ9s*r_pjct{9acGU!Os`h3D&n7-hurf>-shsE%{h$kONY1T(b+^SIG#OwDk&L9mJ))wdol&y2{U+%y?I!b1PLJ-< zW`pjCk7quf1O$g+EnnjYq4;vGaSc0fR#*sZ+nZy5|6E*9S&T;7`$~4 z{son2vK~7A^RfK|3p^SFW)9%21;v3xn!m*Xdck?~&WaQTAdGcgP93u$5ZD8dIS}J; z7Q916r+c1vD-;|ws^|SY;9}hj8RIg7`0Fb`lubT%dTn0Tu))J-&%9Oqx;dd3-0AnE z>tCsv$SJqq>7CqNrvTl8={KreeH0v=)lYvVL^daRrLuqbJWp04)4MjzN((xBD&xCB9=;#t4w2Xv!b$pUT(&K2wLP)dIZ|n~StD;uuDe~>il1!rTY88Y za$L}CN+QUXyS@L=YrF_^+ylVYR2CN1vQW#uYnLr#kI!d{)bV=OP7OGa)mhB$Lt^Y46nq1mCz7O z&D@*&vGjZOQQ4J4?qbqy!mj0|6$w)QqYg>Z@p@V0wJmQx{-fZFG+M9I)XUdEGkxi#!R3sJiqTQ zSGv$xu|J-h?Wpti@f}_DFN=*)ROM-TEtwF%&F`@6308UG#wU&V{x*Ab}mfQ%}oYa%#5cy9!KX(l>WS=WI4@z z1j3*?Z$KaNS%b;_0Z^;_3h^HVnnuJRs36F7Rn= zNt@-xObtLHP2;a3+l{n!igh+(E!XKSjsx6AyjE;bul@erGESJ14J@k=@yz#8sIHRP zor8Rj8tN zgP<`ta|sAd6NB4y4MD^|)kL=Sli4-3`3{I-N(;raHOT%(8Rgmny#tP{V2|^JUQf9* zHtMW4WKtTue0{qd!N}e&&)i}Gc%ug!yAgFl7djXqY*}!=~#GSQG~zNBs@MvB6RYlGFcVLcBDX4xTm5oc0=Lv!FhK?tO&#o+L6%17%CP$SkU z28&r06NEnT)G_JjryAFzbo_{YryP=E3^~i|Zjj%Phvt2oUE2FZjcqgNX&Gc)&I}`= zZZA_{tpm;Yx=WroSz$_8YIsZu@itq*!*VQk+t4Lr8^*A%(QUt3=XSBN%9%NvSz*sc z%R(LUr^B3h$NGviUIMq%%JW#bu$~reX(Tixte}F|bbU3mFO_Vn=yi6q$6)2)mG~1e zdY)%nq@sot37Pz^TOU_~>7pp?gfRDHk=tS{glgSjV-Z0_Sm{M^9@Zec1f+W#1PfT< z=nSZK5xljpgs*wuOaQj{K8)ChyK)Q$sjwK0QO!V0lMu~;9v0ms!O2D-r4&0uUhMIQ zH$d;0X?jCGldyAslxX*w2v$oSQ7Nh%CT(8R;T##ql!oT0czD0n{otJ+Bi>N0LbO9! z+-PdnOa0(Ey>YC7(1L?inBmBKrK^%TJ4t$F#SdQIR2?}mA@}P&5EYkrC&nw}C@ZwQ z1NjI_RQcv=w~pcEmbjMbik;DSBdE1K3w+JIxX*;t8k&R+dU+y!vMcR_4;%-iDEbs; z2je9Z4Ol!>4<#i)V(_9to`xx^2PN9l>n45TyB8@SZ0-Z$wu z6+_el;wq5MI^mpYF=~OwJBbDAVw1uL)!)=uHx)N#sH^!lT^Wz;oj#vb z2fz|gIzw7Dbg~{4OspNqXskVp) z=^jg$>WPkCt!T-_C)ooL+@FRn-cAvP7tLA2GVWamr?rW5la>PeH=bqt{*zYKY?wxVRZS5%|RzQnAX_UuU&-9dz^cHN7C#TE0qqv#{ydsgFb{rZedb&{?}wJN)fzq~Qb1j0*RClBKu ztLMI1W|AYur|G898Ahp<9M4sp@p5Gz2ji19MK{}(N>|ckGkzk& zBSZ&`53D>!-LPv!)N5@88G&WdR-1B4?QJ|!?Bay+UOAqOM(j&pZ2~&$%M|Lahz~_N zoN%l5LkR zK$w%ZAOt!>g@TC0klhh+g@U=|ZKvF1%m;SBzOA`oT37_41U>^OPQwqi0%cHY?I;XB zO1oq)-8f8Z%Dpfkg0uX~J72F_yD8QV^)E@o8+C|mChkZB{j=(84u%z0z?Lm+N96Vf zd4Bv@uiRGidtOht zqUwiR&$i0?S~wxEarDLKt51^<&jsBw1NSl3Bno$giF{wN6YD7^$jGj$FZX<`W@8W@ ztCnI>+OX`j)zcdv43g)sx*%?c1bhX9pJ#hj>D^|$RbetvMnD$Tjv|6?+gh>FQgQH- zwnzL)Wfs^y%MSOsCYuA7uP*hWc7$IC<_!WahL#30hK2-@1jn%6iYo@5fUYiV{;TsSkbh+7l?r z7~r|8luqzFf{^mo3md1=Kk+M0JNaTvd@=Y^VMFl5kEI`Vi~-Xpb}r|c?)8V^o1vG3 zW|0@}EySLp$U~YtB^QE}<`DM;)NQngRpAz4Ixa@z_t(K%3~vkgbL&+VJH5v$*FT%I z7HFAFr7RPhI+L7SK{Vz@p}=J>@wAzW_H=T|y6DSvHLuHJfe1>EalDf9&_HxJX2XHO z=TenPSU(`eJy>L_-`60=j{C-E)da}vigmx4kR+&$2mP_#0xrc zl>|PySh(kZ@XEJUX^Pfu5vljTHFUg*?B6Oo1Te`$$8E}kx;WAe?*mvJvoE^?wZO5C za^>uio;%9n&mOkEk6S0hSjOg{Dv;lKhuN$g=)LC|hZ%o=+A3Zc$NttKT zrxV!cC19MYl5ic=x*LF2UdsS-Hai9j=XgeiO8h6yU;w~_k?(WRy$zk`QE`S+V7%!GpvgRUnb>F3UVQ zO{6q#JI%r6m%b>L1XFL-CKfvz7+l!jE>imvk&WwRaOfNwgmt^h%y9d7EQwCF`_7Wr7aj8AneY0h zWedpmpk#`NXSSdk?B28O3Z2zfvwl?9kmnslUx6$NZdcoThrQXvI_Y2}3X?e&(qpLR zRUW>#G0-`IfQOvk8#Kt#>LcLfqBTRaaM#yhXmFo)<9cx`@d8Nj?BWZk_I~rUB#dGC zS+ejb#-arN(-K`5Z2Z)W7$Oj1u|0a3n>8Cr0&^s!?WpCN#!lZkLtSv&8gl;pg>GB; zIiVh`$n z^oH&-idTo~16E=FhQ=;KEzknAy}tH}KqXCC0A`6V*Us0U5NSZA>$o-$^#laM=9d)NCU}{pres z_**ccfrUDPzl4yGFABlI`$9Un~ z(n?aZlG2GwCOf4$zZ1GT607)72I%j$x&AImG#bi)PA&s9bF#KUlqEIDX+8j8>H~G| z=CPLyoXV%-=P%Z8ae6wHF5=)~rhKnbixQDq*6pLq3q^iRwh|44RIhWDIVHrgh1e4f zAcoPyy3Z1CFXSD(iC~o-Y;{rx#BnQ;2LWfO;)4&{^s!;$YTse4`v!C`nnjc@lH}b; zWh!}yyC$kRWXSg}T%<5PmM1QumoX zfZR}oSFEfdD@DvyRO;9_{cXb_V-k}ks2tM;Ksn6@OCOR8#^o2Rq3bEOSiMfWSTml) z-XN$FL#5le6X-}Mu)c0G*00mB6j==*lJXhzJHw&oCGV4)s?k8-e*pT6iM>ai3ZXp$ z;M7SH5|Y!W>{_OE1wUQsAim|8KM*FUo3AHl+dl=e$J@fNsX+!0)7K47jP91jy4Rq( zZ5!af73&I?(2K>dotkyB;O}Wp-}0_pxZ~Hqkly7XWHWOIE2>PZY;D(VP)L_yHvq`# zi+CtdT$QeUJUikV@UVt>P=NQ z#iS1Fa#7=IIu#Jm$-Az*I#;8@#5l9deC$9C<92SBF|Ih}n=#TD!*5>d&+lUctFX}c zK(d$ObH0@$4c5br$su~VzSP~wA_a3}BDHjk^t2xA3k?l#*D}-%iF;DJ(^U!KxltAM zGH;6;-s68-al5l0q;5SXo0>1jtOCjV%I?(6wf#vgaobDRU>SLH#RD}L2;^i;Dgvv* z+l|t}=3OEDw~rti5s$EZnN`M>fj%|W)Z+CLL7>k)ZzLijOUHoh^rGSgK6G0k(`D^b z_xR`L+S<{4rA!edxcYZd8heBLK-Y$F!dE6hw(52g)|0PGgv?)37gX)|c$Gvv5$oS;hukUyX zj+1?mb4+HlotAF(z}_1RDt0J>KRZAZW)*Te3hcXwKyT~AJ_I3mG6Bs+IalsK2dbIR zkf05nbb@6H{W)beXt!CyxA1=f>>wb*H6l1(oTF^MWmyN&b-y-?>eiPfp7T2LX@iL` z&>g?(Nny=FS&3vD)oeM4n69 z2l6+MCIv6uDKosqbNz5Ni;|ipWn`j{#X@_wg<0YWn$D%?W0oTfK24UR^5hWC32 zbL`G73+Fy<@0<+@<%<%^S02YcP)eze_z)ptucw`4%YP|TEGn{|f}SIBLf*=x;*-B5 z6TjJpUD+YeAY;Av1llQr z+^!8oeWM8n{BA>HVfDDlmRw zlvS^G=+Zau+s^dkIWjT&wF{sdOrl1yri^kce57nV6TT*CM5QwM{T{&R7d9pEGe{sk zcsBMfIl&JA0<5tKsnr6&K%ji+3T-^slG6fiH2j$W`cJ0cwa1y|hL&oxw+H=jq|apt z^lflzS_$43Ft+G`UK)wVmk&4FV&Q$ee1ZON{;RkQ`0mmDdR}?Q`HG{Vt(GNtbm4?- zwnd%JID7FT7)SmK?wbcSK0K1&%Y)|N0&ekystI3_3l`bf+~xz0n*gaBRYD8^HL}fB z3f&f7rcYhzU`liwq3d8a0LH`?U*NUcEu2Lr6lesZuP@JP(8$ZK6Mreyh8^?WB#{z= zF^YM+^J}ptYVKazE2+g^vNhvDj)m1AbvrJFOq=c$OxXlD1z1>vAG?}lt}gnv5~=Vp zlr}tKomDlfdHCMpjb#>TJq8fS@KoMSuG=i9yb^vP{eYHK&|SK=%zzevt%|d`r`cWV zvPq+qEgF-@tZ`T<^G&tJQm$uXR6^A;DaBq%tLiQP*t_WMV$I$$;p^9MmX{VS;xF&0 z!m@fhnGI%G&em=_y5IP=@(bc!tBr{2&U^?4&0u{+T+I%;bF(u0+4TL>YK{AoRnKA_ z->K+^YZlRrDhZ1@t~fPLk?}EkjS93yKS!kH_Dfk})cZ2i^QpzO%#qvg6bjpuOCWTJ ze~*sOah(v(2Hoayv*;MxwFFO*xmsG=)0-$Zw^q=kaNJGuDw%~|Kg zrtAQAmVWh%V14G5r~6d~O`g6O`Z&{>wtL#F$Wg`};mef{i+TPM90WD1~S5*=?o z;>fHN&?^;QmZAdbbWkN6mW0Rp^+4(a#8S+1$rkN7*W=BaSMD|Mz0VH(-N5eBP;SyC zV-$LSD~pzMvi6!aQ&>E|dq@re4+{$mVVChP2i)<~aEX+b)m_4ioB})S%L0jxAGH;O z;8EJ~sO~180aFtxKdTWzf)?7-fuKHbJviTmI)Lg3a3 znqUV2$;@gQp0bd(5hT5!N1R`1)DhUD`Fua8@>nsUVr6PjV+vlr{>r~*pnws3J9?vG z7I35PaQE_ZY1cw?se`|RO#tUZpi&eOXpM~CynJz{%XJ%P>MT%82_5^sZYk=jyc-(J z;|Cn+jLB5?0E`l`ZfOz#q{;gxou4pIE?O3!dN>_<*TARd@_e7yyfUtMR^9e==OWCR z?zM<3H4Nd}L$__9ji4vALFhdhW|!O#9b|)b=Pby^Lv;ZKW2c6q>67I}i@XDKyvWS@t{%e6NWDY^b|Z!dM}uQo%%J zn-677%q#Os{%|gfFQ8=+)URq?)vBmqWHY^NO~`TzUZ{uPfNPmhJ<_RQJbUw*C!4xg zIwGn&m>BzXQIJVd!G=lMxb1;mP7i;uX6}@Z-DZ-D1tusXi^`t>9d}XOzZb><)Ms{! zZUeOg0U@rc5YVap0-~>5&vBFr2*Ca0CMOm}jB{;wvh?;!V}gf z82xa5K+KB)cA1Gs^q`^Lk-ME!>)Mby<*DcjGN~ZxIyU8(jUilmgag z$g3AFGj@9RQwD5_&*j7_a&#uvdD!(Uc6C?$us`jJb3kx(RI~YLOlZ=XE=HvBwxZg= zWP4GprH}0LlReJkfy4GoFT= zDoqo*@AE9+&RCI`DbXc297^Jlya&CFkZ2t~^?t`(E`|L}k@bfzwA_g((^=6wHyw)R4z=}nQq}HqV}%Qu#~)3=gW;}jd(`$ngto{4wp2) zZQZJ~4K3XtHk1=Q{Q0vfkco%g6?;#Mb$x8R==(SlKnC6=!+C<|mZ($tEFkTP!+A}) z5YT+dK1N1H(733eSJJe|YG0xfu!)@pV6VO$Oy}p1t@lI2tp|{zM$(>fx7Og|wd45# z`ENPEeXTUN85AaPpkS2b>2ExiBhr$u$kWlI0>!@1O>ua4khm;}Ji0?&UlgpWPc-p! z*hM=&5U%$#@mxlr;-iX9UVww`sF{SUoU}vU8oVB#%w%jUii^L9Scyjr&+l10D(c-# zpX`kp5+1>zG>^htKmSD21C-hDWWJzzO@^!oVD+k&8@7gJQ>#OJw+ zwv>*hl8CX&B1Qi$Jb!c0dP{!aibAUCY_m^HN($4nXU|j%RcRS@DsxtOv|s^8D1L9GZ0Lm5!L7Qc*}(zA&NQ1KQ5Y-dsnc$Wcp1zHauppqsMiBmbe0U@Qi;yh!V> z_4nW8191bnV3ZBML{Shj4Pes1AI!9VEUf;142Q-Q_ToahE8eboR&_&RE5^+gl)%R} zQ>=DRCM7h7E@sp`^s^^dGuZNvnjSN8si9Ef;qW3L1AVi(v%)s3A;lWS3C#Kp--raB zlH>ef%*$IZo~u#ZT`mn&E5`zbqJ5h29yC~lpqn$qD7i!HxIb9)1IgsiM^f-)EjGla zTsG3xQZ~LNE3_Mi3B()LX-dd?Rd!)n^ey?YD$Ui)ujdIgRf{jq>T8X8)=d0vvy_B? zeHs~03Nsj(ny|Kophplzv3iS$-=(`<&j7hRPT(kVK_)(@6*PXs4~`H5g8b4UMh<8{M57}z2P6(wdK{XyHr)mKP^;EjPcRd~zo!ZC0}PJ87U z0stL2yytZV&>||!0KUC|7!jG8*P5ki1rut~st`CiW9Iz&Bvk2k*&ItZMfD%&al3x{zyAl&Py&iY!;4J%PZIeRd8aaf zfAbwSdHi=h{Kqq|!QcY4*I@bpv40`yF9Zt%CLOAjVjb~6kW-95OgwDN9F0BFKN9nI z$<5IN96yzbwfQF%`pfwF!4wf3tSk-@oHQ^-!2f`}OTd)D2tTW2`8V18i4oEuK(x|< zvCzA}AmdjOy1mRl6X5t9y+Wg_4(Oy(o5Eu!IQp6wTzdr={`*VsIg7JSk z{omjJ=L1v?N<3G{81aAZ{LeRkJp%@(!Gfp@{`?cayf6ba2&1H+^D{X=UrR|jhlWM; z2>x$$|8(IVo;`xQWiQpg{Vpu5xjz(%>90Ka=~=)t0`mPip=xRbfB)vMXONJ1;lPs0 z0e|Je-=8i3p9Y(eQu%L877>9GQ?du5{2N#180XOTM&cy?B*(wf^@oT{1Xzz@V8B;0 z;*Zop0K)^Y)&C;=U)`hxa40{I`QI3hg=LR`6qyD3)g%5)zB>vpoB+2e_P_k@?_u~W vS^j@A41W*3Uun~C3;2s6|G#YkK_Ivan>Svvz4y>zfIk@tMe$Nm!@&On)}+hz literal 0 HcmV?d00001 diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala index 8f50971b74432..d3f9497e1bb5b 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/AggregateFunction.scala @@ -33,7 +33,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation * - merge, and * - resetAccumulator * - * All these methods muse be declared publicly, not static and named exactly as the names + * All these methods must be declared publicly, not static and named exactly as the names * mentioned above. The methods createAccumulator and getValue are defined in the * [[AggregateFunction]] functions, while other methods are explained below. * From b7b0d400a8e630eb81d7ec51f112a4ded5c1b03f Mon Sep 17 00:00:00 2001 From: Xingcan Cui Date: Fri, 1 Sep 2017 09:16:21 +0800 Subject: [PATCH 092/129] [FLINK-7564] [table] Fix watermark semantics in rowtime unbounded OVER window. This closes #4633. --- .../table/runtime/aggregate/RowTimeUnboundedOver.scala | 2 +- .../flink/table/runtime/harness/OverWindowHarnessTest.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala index c8236a3d605f6..27d307b540b32 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/RowTimeUnboundedOver.scala @@ -114,7 +114,7 @@ abstract class RowTimeUnboundedOver( val curWatermark = ctx.timerService().currentWatermark() // discard late record - if (timestamp >= curWatermark) { + if (timestamp > curWatermark) { // ensure every key just registers one timer ctx.timerService.registerEventTimeTimer(curWatermark + 1) 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 ba36e183aa40d..def1972866a07 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 @@ -711,6 +711,9 @@ class OverWindowHarnessTest extends HarnessTestBase{ assert(testHarness.numKeyedStateEntries() == 0) testHarness.processWatermark(20000) + testHarness.processElement(new StreamRecord( + CRow(Row.of(20000L: JLong, "ccc", 1L: JLong), change = true))) // test for late data + testHarness.processElement(new StreamRecord( CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 5000 testHarness.setProcessingTime(2500) @@ -844,6 +847,9 @@ class OverWindowHarnessTest extends HarnessTestBase{ assert(testHarness.numKeyedStateEntries() == 0) testHarness.processWatermark(20000) + testHarness.processElement(new StreamRecord( + CRow(Row.of(20000L: JLong, "ccc", 2L: JLong), change = true))) // test for late data + testHarness.processElement(new StreamRecord( CRow(Row.of(20001L: JLong, "ccc", 1L: JLong), change = true))) // clean-up 5000 testHarness.setProcessingTime(2500) From 0eef8e8c01041bb0c001282a16c43ea54f859cfa Mon Sep 17 00:00:00 2001 From: sunjincheng121 Date: Tue, 5 Sep 2017 08:55:03 +0800 Subject: [PATCH 093/129] [FLINK-7572] [table] Improve TableSchema and FlinkTable validation exception messages. This closes #4640. --- .../apache/flink/table/api/TableSchema.scala | 15 ++++++- .../flink/table/plan/schema/FlinkTable.scala | 15 ++++++- .../validation/FlinkTableValidationTest.scala | 39 +++++++++++++++++++ .../TableSchemaValidationTest.scala | 27 ++++++++++++- 4 files changed, 90 insertions(+), 6 deletions(-) create mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/FlinkTableValidationTest.scala 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 a67a07af106f0..6ee65f0a5c6da 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 @@ -28,13 +28,24 @@ class TableSchema( if (columnNames.length != columnTypes.length) { throw new TableException( - "Number of column indexes and column names must be equal.") + s"Number of field names and field types must be equal.\n" + + s"Number of names is ${columnNames.length}, number of types is ${columnTypes.length}.\n" + + s"List of field names: ${columnNames.mkString("[", ", ", "]")}.\n" + + s"List of field types: ${columnTypes.mkString("[", ", ", "]")}.") } // check uniqueness of field names if (columnNames.toSet.size != columnTypes.length) { + val duplicateFields = columnNames + // count occurences of field names + .groupBy(identity).mapValues(_.length) + // filter for occurences > 1 and map to field name + .filter(g => g._2 > 1).keys + throw new TableException( - "Table column names must be unique.") + s"Field names must be unique.\n" + + s"List of duplicate fields: ${duplicateFields.mkString("[", ", ", "]")}.\n" + + s"List of all fields: ${columnNames.mkString("[", ", ", "]")}.") } val columnNameToIndex: Map[String, Int] = columnNames.zipWithIndex.toMap diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala index df56ae6ec2405..c76532fceaa10 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala @@ -37,13 +37,24 @@ abstract class FlinkTable[T]( if (fieldIndexes.length != fieldNames.length) { throw new TableException( - "Number of field indexes and field names must be equal.") + s"Number of field names and field indexes must be equal.\n" + + s"Number of names is ${fieldNames.length}, number of indexes is ${fieldIndexes.length}.\n" + + s"List of column names: ${fieldNames.mkString("[", ", ", "]")}.\n" + + s"List of column indexes: ${fieldIndexes.mkString("[", ", ", "]")}.") } // check uniqueness of field names if (fieldNames.length != fieldNames.toSet.size) { + val duplicateFields = fieldNames + // count occurences of field names + .groupBy(identity).mapValues(_.length) + // filter for occurences > 1 and map to field name + .filter(g => g._2 > 1).keys + throw new TableException( - "Table field names must be unique.") + s"Field names must be unique.\n" + + s"List of duplicate fields: ${duplicateFields.mkString("[", ", ", "]")}.\n" + + s"List of all fields: ${fieldNames.mkString("[", ", ", "]")}.") } val fieldTypes: Array[TypeInformation[_]] = diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/FlinkTableValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/FlinkTableValidationTest.scala new file mode 100644 index 0000000000000..a845f5c1f1bbf --- /dev/null +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/FlinkTableValidationTest.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.api.validation + +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.TableException +import org.apache.flink.table.api.scala._ +import org.apache.flink.table.utils.TableTestBase +import org.junit.Test + +class FlinkTableValidationTest extends TableTestBase { + + @Test + def testFieldNamesDuplicate() { + + thrown.expect(classOf[TableException]) + thrown.expectMessage("Field names must be unique.\n" + + "List of duplicate fields: [a].\n" + + "List of all fields: [a, a, b].") + + val util = batchTestUtil() + util.addTable[(Int, Int, String)]("MyTable", 'a, 'a, 'b) + } +} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/TableSchemaValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/TableSchemaValidationTest.scala index 1a7815aa1b257..c430e59efa0c1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/TableSchemaValidationTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/validation/TableSchemaValidationTest.scala @@ -24,12 +24,35 @@ import org.junit.Test class TableSchemaValidationTest extends TableTestBase { - @Test(expected = classOf[TableException]) - def testInvalidSchema() { + @Test + def testColumnNameAndColumnTypeNotEqual() { + thrown.expect(classOf[TableException]) + thrown.expectMessage( + "Number of field names and field types must be equal.\n" + + "Number of names is 3, number of types is 2.\n" + + "List of field names: [a, b, c].\n" + + "List of field types: [Integer, String].") + val fieldNames = Array("a", "b", "c") val typeInfos: Array[TypeInformation[_]] = Array( BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO) new TableSchema(fieldNames, typeInfos) } + + @Test + def testColumnNamesDuplicate() { + thrown.expect(classOf[TableException]) + thrown.expectMessage( + "Field names must be unique.\n" + + "List of duplicate fields: [a].\n" + + "List of all fields: [a, a, c].") + + val fieldNames = Array("a", "a", "c") + val typeInfos: Array[TypeInformation[_]] = Array( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO) + new TableSchema(fieldNames, typeInfos) + } } From 4cf737c1c12a4d61e9388992ee96875c43faa410 Mon Sep 17 00:00:00 2001 From: Fabian Hueske Date: Wed, 9 Aug 2017 17:41:10 +0200 Subject: [PATCH 094/129] [FLINK-7404] [table] Generate code for non-equi join conditions only. This closes #4507. --- .../apache/flink/table/plan/nodes/dataset/DataSetJoin.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetJoin.scala index 1583e31589d16..acbf94dd6516a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetJoin.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetJoin.scala @@ -186,7 +186,8 @@ class DataSetJoin( |""".stripMargin } else { - val condition = generator.generateExpression(joinCondition) + val nonEquiPredicates = joinInfo.getRemaining(this.cluster.getRexBuilder) + val condition = generator.generateExpression(nonEquiPredicates) body = s""" |${condition.code} |if (${condition.resultTerm}) { From 7c11bd7f4127bc550a8fd17fc0fd843b929e3cb2 Mon Sep 17 00:00:00 2001 From: Usman Younas Date: Mon, 28 Aug 2017 13:44:02 +0000 Subject: [PATCH 095/129] [FLINK-7227] [table] Fix push-down of disjunctive predicates with more than two terms. This closes #4608. --- .../table/plan/util/RexProgramExtractor.scala | 7 +- .../table/plan/RexProgramExtractorTest.scala | 73 +++++++++++++++++-- .../utils/TestFilterableTableSource.scala | 1 + 3 files changed, 72 insertions(+), 9 deletions(-) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala index bf9a6881addef..53bf8e777af33 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/RexProgramExtractor.scala @@ -20,10 +20,11 @@ package org.apache.flink.table.plan.util import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.rex._ +import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.sql.{SqlFunction, SqlPostfixOperator} import org.apache.flink.table.api.TableException import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.expressions.{Expression, Literal, ResolvedFieldReference} +import org.apache.flink.table.expressions.{And, Expression, Literal, Or, ResolvedFieldReference} import org.apache.flink.table.validate.FunctionCatalog import org.apache.flink.util.Preconditions @@ -170,6 +171,10 @@ class RexNodeToExpressionConverter( None } else { call.getOperator match { + case SqlStdOperatorTable.OR => + Option(operands.reduceLeft(Or)) + case SqlStdOperatorTable.AND => + Option(operands.reduceLeft(And)) case function: SqlFunction => lookupFunction(replace(function.getName), operands) case postfix: SqlPostfixOperator => diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala index 840be171614b1..c2a01c68ad441 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/RexProgramExtractorTest.scala @@ -20,12 +20,13 @@ package org.apache.flink.table.plan import java.math.BigDecimal -import org.apache.calcite.rex.{RexBuilder, RexProgram, RexProgramBuilder} +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rex._ import org.apache.calcite.sql.SqlPostfixOperator import org.apache.calcite.sql.`type`.SqlTypeName.{BIGINT, INTEGER, VARCHAR} import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.flink.table.expressions._ -import org.apache.flink.table.plan.util.RexProgramExtractor +import org.apache.flink.table.plan.util.{RexNodeToExpressionConverter, RexProgramExtractor} import org.apache.flink.table.utils.InputTypeBuilder.inputOf import org.apache.flink.table.validate.FunctionCatalog import org.hamcrest.CoreMatchers.is @@ -33,6 +34,7 @@ import org.junit.Assert.{assertArrayEquals, assertEquals, assertThat} import org.junit.Test import scala.collection.JavaConverters._ +import scala.collection.mutable class RexProgramExtractorTest extends RexProgramTestBase { @@ -104,6 +106,8 @@ class RexProgramExtractorTest extends RexProgramTestBase { val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) // 100 val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + // 200 + val t4 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(200L)) // a = amount < 100 val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t3)) @@ -113,15 +117,17 @@ class RexProgramExtractorTest extends RexProgramTestBase { val c = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t3)) // d = amount <= id val d = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + // e = price == 200 + val e = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t4)) // a AND b val and = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(a, b).asJava)) - // (a AND b) or c - val or = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.OR, List(and, c).asJava)) - // not d + // (a AND b) OR c OR e + val or = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.OR, List(and, c, e).asJava)) + // NOT d val not = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.NOT, List(d).asJava)) - // (a AND b) OR c) AND (NOT d) + // (a AND b) OR c OR e) AND (NOT d) builder.addCondition(builder.addExpr( rexBuilder.makeCall(SqlStdOperatorTable.AND, List(or, not).asJava))) @@ -134,13 +140,64 @@ class RexProgramExtractorTest extends RexProgramTestBase { functionCatalog) val expected: Array[Expression] = Array( - ExpressionParser.parseExpression("amount < 100 || price == 100"), - ExpressionParser.parseExpression("id > 100 || price == 100"), + ExpressionParser.parseExpression("amount < 100 || price == 100 || price === 200"), + ExpressionParser.parseExpression("id > 100 || price == 100 || price === 200"), ExpressionParser.parseExpression("!(amount <= id)")) assertExpressionArrayEquals(expected, convertedExpressions) assertEquals(0, unconvertedRexNodes.length) } + @Test + def testExtractANDExpressions(): Unit = { + val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) + val builder = new RexProgramBuilder(inputRowType, rexBuilder) + + // amount + val t0 = rexBuilder.makeInputRef(allFieldTypes.get(2), 2) + // id + val t1 = rexBuilder.makeInputRef(allFieldTypes.get(1), 1) + // price + val t2 = rexBuilder.makeInputRef(allFieldTypes.get(3), 3) + // 100 + val t3 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L)) + + // a = amount < 100 + val a = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t0, t3)) + // b = id > 100 + val b = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t3)) + // c = price == 100 + val c = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, t2, t3)) + // d = amount <= id + val d = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, t0, t1)) + + // a AND b AND c AND d + val and = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(a, b, c, d).asJava)) + + builder.addCondition(builder.addExpr(and)) + + val program = builder.getProgram + val relBuilder: RexBuilder = new RexBuilder(typeFactory) + + val expanded = program.expandLocalRef(program.getCondition) + + var convertedExpressions = new mutable.ArrayBuffer[Expression] + val unconvertedRexNodes = new mutable.ArrayBuffer[RexNode] + val inputNames = program.getInputRowType.getFieldNames.asScala.toArray + val converter = new RexNodeToExpressionConverter(inputNames, functionCatalog) + + expanded.accept(converter) match { + case Some(expression) => + convertedExpressions += expression + case None => unconvertedRexNodes += expanded + } + + val expected: Array[Expression] = Array( + ExpressionParser.parseExpression("amount < 100 && id > 100 && price === 100 && amount <= id")) + + assertExpressionArrayEquals(expected, convertedExpressions.toArray) + assertEquals(0, unconvertedRexNodes.length) + } + @Test def testExtractArithmeticConditions(): Unit = { val inputRowType = typeFactory.createStructType(allFieldTypes, allFieldNames) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala index dcf2acd3766b2..fb99864db3ee0 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TestFilterableTableSource.scala @@ -89,6 +89,7 @@ class TestFilterableTableSource( iterator.remove() case (_, _) => } + case _ => } } From 9fe8f217a69c67abd81f1424cc62cbb3d35c25c7 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 1 Sep 2017 12:20:18 +0200 Subject: [PATCH 096/129] [FLINK-7568] Change role of ProcessWindowFunction and WindowFunction in doc --- docs/dev/stream/operators/windows.md | 257 ++++++++++++++------------- 1 file changed, 129 insertions(+), 128 deletions(-) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index c2d557f444cfa..4cfebf2275a7d 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -460,118 +460,15 @@ The above example appends all input `Long` values to an initially empty `String` Attention `fold()` cannot be used with session windows or other mergeable windows. -### WindowFunction - The Generic Case - -A `WindowFunction` gets an `Iterable` containing all the elements of the window and provides -the most flexibility of all window functions. This comes -at the cost of performance and resource consumption, because elements cannot be incrementally -aggregated but instead need to be buffered internally until the window is considered ready for processing. - -The signature of a `WindowFunction` looks as follows: - -
-
-{% highlight java %} -public interface WindowFunction extends Function, Serializable { - - /** - * Evaluates the window and outputs none or several elements. - * - * @param key The key for which this window is evaluated. - * @param window The window that is being evaluated. - * @param input The elements in the window being evaluated. - * @param out A collector for emitting elements. - * - * @throws Exception The function may throw exceptions to fail the program and trigger recovery. - */ - void apply(KEY key, W window, Iterable input, Collector out) throws Exception; -} -{% endhighlight %} -
- -
-{% highlight scala %} -trait WindowFunction[IN, OUT, KEY, W <: Window] extends Function with Serializable { - - /** - * Evaluates the window and outputs none or several elements. - * - * @param key The key for which this window is evaluated. - * @param window The window that is being evaluated. - * @param input The elements in the window being evaluated. - * @param out A collector for emitting elements. - * @throws Exception The function may throw exceptions to fail the program and trigger recovery. - */ - def apply(key: KEY, window: W, input: Iterable[IN], out: Collector[OUT]) -} -{% endhighlight %} -
-
- -A `WindowFunction` can be defined and used like this: - -
-
-{% highlight java %} -DataStream> input = ...; - -input - .keyBy() - .window() - .apply(new MyWindowFunction()); - -/* ... */ - -public class MyWindowFunction implements WindowFunction, String, String, TimeWindow> { - - void apply(String key, TimeWindow window, Iterable> input, Collector out) { - long count = 0; - for (Tuple in: input) { - count++; - } - out.collect("Window: " + window + "count: " + count); - } -} - -{% endhighlight %} -
- -
-{% highlight scala %} -val input: DataStream[(String, Long)] = ... - -input - .keyBy() - .window() - .apply(new MyWindowFunction()) - -/* ... */ - -class MyWindowFunction extends WindowFunction[(String, Long), String, String, TimeWindow] { - - def apply(key: String, window: TimeWindow, input: Iterable[(String, Long)], out: Collector[String]): () = { - var count = 0L - for (in <- input) { - count = count + 1 - } - out.collect(s"Window $window count: $count") - } -} -{% endhighlight %} -
-
- -The example shows a `WindowFunction` to count the elements in a window. In addition, the window function adds information about the window to the output. - -Attention Note that using `WindowFunction` for simple aggregates such as count is quite inefficient. The next section shows how a `ReduceFunction` can be combined with a `WindowFunction` to get both incremental aggregation and the added information of a `WindowFunction`. - ### ProcessWindowFunction -In places where a `WindowFunction` can be used you can also use a `ProcessWindowFunction`. This -is very similar to `WindowFunction`, except that the interface allows to query more information -about the context in which the window evaluation happens. +A ProcessWindowFunction gets an Iterable containing all the elements of the window, and a Context +object with access to time and state information, which enables it to provide more flexibility than +other window functions. This comes at the cost of performance and resource consumption, because +elements cannot be incrementally aggregated but instead need to be buffered internally until the +window is considered ready for processing. -This is the `ProcessWindowFunction` interface: +The signature of `ProcessWindowFunction` looks as follows:
@@ -620,7 +517,6 @@ abstract class ProcessWindowFunction[IN, OUT, KEY, W <: Window] extends Function * @param out A collector for emitting elements. * @throws Exception The function may throw exceptions to fail the program and trigger recovery. */ - @throws[Exception] def process( key: KEY, context: Context, @@ -641,7 +537,7 @@ abstract class ProcessWindowFunction[IN, OUT, KEY, W <: Window] extends Function
-It can be used like this: +A `ProcessWindowFunction` can be defined and used like this:
@@ -652,6 +548,20 @@ input .keyBy() .window() .process(new MyProcessWindowFunction()); + +/* ... */ + +public class MyProcessWindowFunction implements ProcessWindowFunction, String, String, TimeWindow> { + + void process(String key, Context context, Iterable> input, Collector out) { + long count = 0; + for (Tuple in: input) { + count++; + } + out.collect("Window: " + context.window() + "count: " + count); + } +} + {% endhighlight %}
@@ -663,25 +573,42 @@ input .keyBy() .window() .process(new MyProcessWindowFunction()) + +/* ... */ + +class MyWindowFunction extends ProcessWindowFunction[(String, Long), String, String, TimeWindow] { + + def apply(key: String, context: Context, input: Iterable[(String, Long)], out: Collector[String]): () = { + var count = 0L + for (in <- input) { + count = count + 1 + } + out.collect(s"Window ${context.window} count: $count") + } +} {% endhighlight %}
-### WindowFunction with Incremental Aggregation +The example shows a `ProcessWindowFunction` that counts the elements in a window. In addition, the window function adds information about the window to the output. + +Attention Note that using `ProcessWindowFunction` for simple aggregates such as count is quite inefficient. The next section shows how a `ReduceFunction` can be combined with a `ProcessWindowFunction` to get both incremental aggregation and the added information of a `ProcessWindowFunction`. -A `WindowFunction` can be combined with either a `ReduceFunction` or a `FoldFunction` to +### ProcessWindowFunction with Incremental Aggregation + +A `ProcessWindowFunction` can be combined with either a `ReduceFunction` or a `FoldFunction` to incrementally aggregate elements as they arrive in the window. -When the window is closed, the `WindowFunction` will be provided with the aggregated result. +When the window is closed, the `ProcessWindowFunction` will be provided with the aggregated result. This allows to incrementally compute windows while having access to the -additional window meta information of the `WindowFunction`. +additional window meta information of the `ProcessWindowFunction`. -Note You can also `ProcessWindowFunction` instead of -`WindowFunction` for incremental window aggregation. +Note You can also the legacy `WindowFunction` instead of +`ProcessWindowFunction` for incremental window aggregation. #### Incremental Window Aggregation with FoldFunction The following example shows how an incremental `FoldFunction` can be combined with -a `WindowFunction` to extract the number of events in the window and return also +a `ProcessWindowFunction` to extract the number of events in the window and return also the key and end time of the window.
@@ -692,7 +619,7 @@ DataStream input = ...; input .keyBy() .timeWindow() - .fold(new Tuple3("",0L, 0), new MyFoldFunction(), new MyWindowFunction()) + .fold(new Tuple3("",0L, 0), new MyFoldFunction(), new MyProcessWindowFunction()) // Function definitions @@ -706,15 +633,15 @@ private static class MyFoldFunction } } -private static class MyWindowFunction - implements WindowFunction, Tuple3, String, TimeWindow> { +private static class MyProcessWindowFunction + implements ProcessWindowFunction, Tuple3, String, TimeWindow> { - public void apply(String key, - TimeWindow window, + public void process(String key, + Context context, Iterable> counts, Collector> out) { Integer count = counts.iterator().next().getField(2); - out.collect(new Tuple3(key, window.getEnd(),count)); + out.collect(new Tuple3(key, context.window().getEnd(),count)); } } @@ -759,7 +686,7 @@ DataStream input = ...; input .keyBy() .timeWindow() - .reduce(new MyReduceFunction(), new MyWindowFunction()); + .reduce(new MyReduceFunction(), new MyProcessWindowFunction()); // Function definitions @@ -770,11 +697,11 @@ private static class MyReduceFunction implements ReduceFunction { } } -private static class MyWindowFunction - implements WindowFunction, String, TimeWindow> { +private static class MyProcessWindowFunction + implements ProcessWindowFunction, String, TimeWindow> { public void apply(String key, - TimeWindow window, + Context context, Iterable minReadings, Collector> out) { SensorReading min = minReadings.iterator().next(); @@ -808,6 +735,80 @@ input
+### WindowFunction (Legacy) + +In some places where a `ProcessWindowFunction` can be used you can also use a `WindowFunction`. This +is an older version of `ProcessWindowFunction` that provides less contextual information and does +not have some advances features, such as per-window keyed state. This interface will be deprecated +at some point. + +The signature of a `WindowFunction` looks as follows: + +
+
+{% highlight java %} +public interface WindowFunction extends Function, Serializable { + + /** + * Evaluates the window and outputs none or several elements. + * + * @param key The key for which this window is evaluated. + * @param window The window that is being evaluated. + * @param input The elements in the window being evaluated. + * @param out A collector for emitting elements. + * + * @throws Exception The function may throw exceptions to fail the program and trigger recovery. + */ + void apply(KEY key, W window, Iterable input, Collector out) throws Exception; +} +{% endhighlight %} +
+ +
+{% highlight scala %} +trait WindowFunction[IN, OUT, KEY, W <: Window] extends Function with Serializable { + + /** + * Evaluates the window and outputs none or several elements. + * + * @param key The key for which this window is evaluated. + * @param window The window that is being evaluated. + * @param input The elements in the window being evaluated. + * @param out A collector for emitting elements. + * @throws Exception The function may throw exceptions to fail the program and trigger recovery. + */ + def apply(key: KEY, window: W, input: Iterable[IN], out: Collector[OUT]) +} +{% endhighlight %} +
+
+ +It can be used like this: + +
+
+{% highlight java %} +DataStream> input = ...; + +input + .keyBy() + .window() + .apply(new MyWindowFunction()); +{% endhighlight %} +
+ +
+{% highlight scala %} +val input: DataStream[(String, Long)] = ... + +input + .keyBy() + .window() + .apply(new MyWindowFunction()) +{% endhighlight %} +
+
+ ## Triggers A `Trigger` determines when a window (as formed by the *window assigner*) is ready to be From 006572f4031eaaca586f789f103297bbc10dab77 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 1 Sep 2017 12:28:13 +0200 Subject: [PATCH 097/129] [FLINK-7568] Update ProcessFunction.Context in window documentation --- docs/dev/stream/operators/windows.md | 63 +++++++++++++++++++++++----- 1 file changed, 53 insertions(+), 10 deletions(-) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 4cfebf2275a7d..63113b67dbada 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -491,15 +491,35 @@ public abstract class ProcessWindowFunction impl Iterable elements, Collector out) throws Exception; - /** - * The context holding window metadata - */ - public abstract class Context { - /** - * @return The window that is being evaluated. - */ - public abstract W window(); - } + /** + * The context holding window metadata. + */ + public abstract class Context implements java.io.Serializable { + /** + * Returns the window that is being evaluated. + */ + public abstract W window(); + + /** Returns the current processing time. */ + public abstract long currentProcessingTime(); + + /** Returns the current event-time watermark. */ + public abstract long currentWatermark(); + + /** + * State accessor for per-key and per-window state. + * + *

NOTE:If you use per-window state you have to ensure that you clean it up + * by implementing {@link ProcessWindowFunction#clear(Context)}. + */ + public abstract KeyedStateStore windowState(); + + /** + * State accessor for per-key global state. + */ + public abstract KeyedStateStore globalState(); + } + } {% endhighlight %} @@ -528,15 +548,38 @@ abstract class ProcessWindowFunction[IN, OUT, KEY, W <: Window] extends Function */ abstract class Context { /** - * @return The window that is being evaluated. + * Returns the window that is being evaluated. */ def window: W + + /** + * Returns the current processing time. + */ + def currentProcessingTime: Long + + /** + * Returns the current event-time watermark. + */ + def currentWatermark: Long + + /** + * State accessor for per-key and per-window state. + */ + def windowState: KeyedStateStore + + /** + * State accessor for per-key global state. + */ + def globalState: KeyedStateStore } + } {% endhighlight %} + + A `ProcessWindowFunction` can be defined and used like this:

From f35f2d63563f259056b739d67c8ef6ce52167f86 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 1 Sep 2017 12:33:40 +0200 Subject: [PATCH 098/129] [FLINK-7568] Add note about 'key' parameter to window doc --- docs/dev/stream/operators/windows.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 63113b67dbada..1bd6dd9b31cb9 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -578,7 +578,10 @@ abstract class ProcessWindowFunction[IN, OUT, KEY, W <: Window] extends Function
- +Note The `key` parameter is the key that is extracted +via the `KeySelector` that was specified for the `keyBy()` invocation. In case of tuple-index +keys or string-field references this key type is always `Tuple` and you have to manually cast +it to a tuple of the correct size to extract the key fields. A `ProcessWindowFunction` can be defined and used like this: From 7d9e3bf8bc606fb4ac28a17f26df476f86d20c3e Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 1 Sep 2017 13:45:01 +0200 Subject: [PATCH 099/129] [FLINK-7568] Add note about start/end timestamps in window doc --- docs/dev/stream/operators/windows.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 1bd6dd9b31cb9..85a6630dc39c9 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -111,6 +111,11 @@ windows) assign elements to windows based on time, which can either be processin time. Please take a look at our section on [event time]({{ site.baseurl }}/dev/event_time.html) to learn about the difference between processing time and event time and how timestamps and watermarks are generated. +Time-based windows have a *start timestamp* (inclusive) and an *end timestamp* (exclusive) +that together describe the size of the window. In code, Flink uses `TimeWindow` when working with +time-based windows which has methods for querying the start- and end-timestamp and also an +additional method `maxTimestamp()` that returns the largest allowed timestamp for a given windows. + In the following, we show how Flink's pre-defined window assigners work and how they are used in a DataStream program. The following figures visualize the workings of each assigner. The purple circles represent elements of the stream, which are partitioned by some key (in this case *user 1*, *user 2* and *user 3*). From 6d2124ee2e9a1d629bc56b470f356faa90559e1c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Fri, 1 Sep 2017 15:29:21 +0200 Subject: [PATCH 100/129] [FLINK-7568] Add section about consecutive windows to window doc --- docs/dev/stream/operators/windows.md | 74 ++++++++++++++++++++++++++++ 1 file changed, 74 insertions(+) diff --git a/docs/dev/stream/operators/windows.md b/docs/dev/stream/operators/windows.md index 85a6630dc39c9..012d5313742bd 100644 --- a/docs/dev/stream/operators/windows.md +++ b/docs/dev/stream/operators/windows.md @@ -1080,6 +1080,80 @@ as they may "bridge" the gap between two pre-existing, unmerged windows. Attention You should be aware that the elements emitted by a late firing should be treated as updated results of a previous computation, i.e., your data stream will contain multiple results for the same computation. Depending on your application, you need to take these duplicated results into account or deduplicate them. +## Working with window results + +The result of a windowed operation is again a `DataStream`, no information about the windowed +operations is retained in the result elements so if you want to keep meta-information about the +window you have to manually encode that information in the result elements in your +`ProcessWindowFunction`. The only relevant information that is set on the result elements is the +element *timestamp*. This is set to the maximum allowed timestamp of the processed window, which +is *end timestamp - 1*, since the window-end timestamp is exclusive. Note that this is true for both +event-time windows and processing-time windows. i.e. after a windowed operations elements always +have a timestamp, but this can be an event-time timestamp or a processing-time timestamp. For +processing-time windows this has no special implications but for event-time windows this together +with how watermarks interact with windows enables +[consecutive windowed operations](#consecutive-windowed-operations) with the same window sizes. We +will cover this after taking a look how watermarks interact with windows. + +### Interaction of watermarks and windows + +Before continuing in this section you might want to take a look at our section about +[event time and watermarks]({{ site.baseurl }}/dev/event_time.html). + +When watermarks arrive at the window operator this triggers two things: + - the watermark triggers computation of all windows where the maximum timestamp (which is + *end-timestamp - 1*) is smaller than the new watermark + - the watermark is forwarded (as is) to downstream operations + +Intuitively, a watermark "flushes" out any windows that would be considered late in downstream +operations once they receive that watermark. + +### Consecutive windowed operations + +As mentioned before, the way the timestamp of windowed results is computed and how watermarks +interact with windows allows stringing together consecutive windowed operations. This can be useful +when you want to do two consecutive windowed operations where you want to use different keys but +still want elements from the same upstream window to end up in the same downstream window. Consider +this example: + +
+
+{% highlight java %} +DataStream input = ...; + +DataStream resultsPerKey = input + .keyBy() + .window(TumblingEventTimeWindows.of(Time.seconds(5))) + .reduce(new Summer()); + +DataStream globalResults = resultsPerKey + .windowAll(TumblingEventTimeWindows.of(Time.seconds(5))) + .process(new TopKWindowFunction()); + +{% endhighlight %} +
+ +
+{% highlight scala %} +val input: DataStream[Int] = ... + +val resultsPerKey = input + .keyBy() + .window(TumblingEventTimeWindows.of(Time.seconds(5))) + .reduce(new Summer()) + +val globalResults = resultsPerKey + .windowAll(TumblingEventTimeWindows.of(Time.seconds(5))) + .process(new TopKWindowFunction()) +{% endhighlight %} +
+
+ +In this example, the results for time window `[0, 5)` from the first operation will also end up in +time window `[0, 5)` in the subsequent windowed operation. This allows calculating a sum per key +and then calculating the top-k elements within the same window in the second operation. +and then calculating the top-k elements within the same window in the second operation. + ## Useful state size considerations Windows can be defined over long periods of time (such as days, weeks, or months) and therefore accumulate very large state. There are a couple of rules to keep in mind when estimating the storage requirements of your windowing computation: From 769ce2a07424712d62b6ead87c10aee08d85c216 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 4 Sep 2017 16:42:24 +0200 Subject: [PATCH 101/129] [FLINK-7576] [futures] Add FutureUtils.retryWithDelay FutureUtils.retryWithDelay executes the given operation of type Callable> n times and waits in between retries the given delay. This allows to retry an operation with a specified delay. Make retry and retry with delay future properly cancellable This closes #4637. --- .../flink/runtime/concurrent/FutureUtils.java | 171 +++++++-- .../concurrent/ConjunctFutureTest.java | 246 ++++++++++++ .../runtime/concurrent/FutureUtilsTest.java | 359 +++++++++--------- 3 files changed, 554 insertions(+), 222 deletions(-) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ConjunctFutureTest.java 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 5c6439d3800f6..b982c8e86f6c7 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 @@ -26,13 +26,15 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.concurrent.Callable; +import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.function.Supplier; import scala.concurrent.Future; import scala.concurrent.duration.FiniteDuration; @@ -48,6 +50,7 @@ public class FutureUtils { // retrying operations // ------------------------------------------------------------------------ + /** * Retry the given operation the given number of times in case of a failure. * @@ -58,35 +61,135 @@ public class FutureUtils { * @return Future containing either the result of the operation or a {@link RetryException} */ public static CompletableFuture retry( - final Callable> operation, - final int retries, - final Executor executor) { + final Supplier> operation, + final int retries, + final Executor executor) { + + final CompletableFuture resultFuture = new CompletableFuture<>(); - CompletableFuture operationResultFuture; + retryOperation(resultFuture, operation, retries, executor); + + return resultFuture; + } - try { - operationResultFuture = operation.call(); - } catch (Exception e) { - return FutureUtils.completedExceptionally(new RetryException("Could not execute the provided operation.", e)); + /** + * Helper method which retries the provided operation in case of a failure. + * + * @param resultFuture to complete + * @param operation to retry + * @param retries until giving up + * @param executor to run the futures + * @param type of the future's result + */ + private static void retryOperation( + final CompletableFuture resultFuture, + final Supplier> operation, + final int retries, + final Executor executor) { + + if (!resultFuture.isDone()) { + final CompletableFuture operationFuture = operation.get(); + + operationFuture.whenCompleteAsync( + (t, throwable) -> { + if (throwable != null) { + if (throwable instanceof CancellationException) { + resultFuture.completeExceptionally(new RetryException("Operation future was cancelled.", throwable)); + } else { + if (retries > 0) { + retryOperation( + resultFuture, + operation, + retries - 1, + executor); + } else { + resultFuture.completeExceptionally(new RetryException("Could not complete the operation. Number of retries " + + "has been exhausted.", throwable)); + } + } + } else { + resultFuture.complete(t); + } + }, + executor); + + resultFuture.whenComplete( + (t, throwable) -> operationFuture.cancel(false)); } + } + + /** + * Retry the given operation with the given delay in between failures. + * + * @param operation to retry + * @param retries number of retries + * @param retryDelay delay between retries + * @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 of failures + */ + public static CompletableFuture retryWithDelay( + final Supplier> operation, + final int retries, + final Time retryDelay, + final ScheduledExecutor scheduledExecutor) { + + final CompletableFuture resultFuture = new CompletableFuture<>(); + + retryOperationWithDelay( + resultFuture, + operation, + retries, + retryDelay, + scheduledExecutor); + + return resultFuture; + } - return operationResultFuture.handleAsync( - (t, throwable) -> { - if (throwable != null) { - if (retries > 0) { - return retry(operation, retries - 1, executor); + private static void retryOperationWithDelay( + final CompletableFuture resultFuture, + final Supplier> operation, + final int retries, + final Time retryDelay, + final ScheduledExecutor scheduledExecutor) { + + if (!resultFuture.isDone()) { + final CompletableFuture operationResultFuture = operation.get(); + + operationResultFuture.whenCompleteAsync( + (t, throwable) -> { + if (throwable != null) { + if (throwable instanceof CancellationException) { + resultFuture.completeExceptionally(new RetryException("Operation future was cancelled.", throwable)); + } else { + if (retries > 0) { + final ScheduledFuture scheduledFuture = scheduledExecutor.schedule( + () -> retryOperationWithDelay(resultFuture, operation, retries - 1, retryDelay, scheduledExecutor), + retryDelay.toMilliseconds(), + TimeUnit.MILLISECONDS); + + resultFuture.whenComplete( + (innerT, innerThrowable) -> scheduledFuture.cancel(false)); + } else { + resultFuture.completeExceptionally(new RetryException("Could not complete the operation. Number of retries " + + "has been exhausted.", throwable)); + } + } } else { - return FutureUtils.completedExceptionally(new RetryException("Could not complete the operation. Number of retries " + - "has been exhausted.", throwable)); + resultFuture.complete(t); } - } else { - return CompletableFuture.completedFuture(t); - } - }, - executor) - .thenCompose(value -> value); + }, + scheduledExecutor); + + resultFuture.whenComplete( + (t, throwable) -> operationResultFuture.cancel(false)); + } } + /** + * Exception with which the returned future is completed if the {@link #retry(Supplier, int, Executor)} + * operation fails. + */ public static class RetryException extends Exception { private static final long serialVersionUID = 3613470781274141862L; @@ -109,14 +212,14 @@ public RetryException(Throwable cause) { // ------------------------------------------------------------------------ /** - * Creates a future that is complete once multiple other futures completed. + * Creates a future that is complete once multiple other futures completed. * The future fails (completes exceptionally) once one of the futures in the * conjunction fails. Upon successful completion, the future returns the * collection of the futures' results. * *

The ConjunctFuture gives access to how many Futures in the conjunction have already - * completed successfully, via {@link ConjunctFuture#getNumFuturesCompleted()}. - * + * completed successfully, via {@link ConjunctFuture#getNumFuturesCompleted()}. + * * @param futures The futures that make up the conjunction. No null entries are allowed. * @return The ConjunctFuture that completes once all given futures are complete (or one fails). */ @@ -158,7 +261,7 @@ public static ConjunctFuture waitForAll(CollectionThe advantage of using the ConjunctFuture over chaining all the futures (such as via * {@link CompletableFuture#thenCombine(CompletionStage, BiFunction)} )}) is that ConjunctFuture * also tracks how many of the Futures are already complete. @@ -183,16 +286,16 @@ public abstract static class ConjunctFuture extends CompletableFuture { */ private static class ResultConjunctFuture extends ConjunctFuture> { - /** The total number of futures in the conjunction */ + /** The total number of futures in the conjunction. */ private final int numTotal; - /** The next free index in the results arrays */ + /** The next free index in the results arrays. */ private final AtomicInteger nextIndex = new AtomicInteger(0); - /** The number of futures in the conjunction that are already complete */ + /** The number of futures in the conjunction that are already complete. */ private final AtomicInteger numCompleted = new AtomicInteger(0); - /** The set of collected results so far */ + /** The set of collected results so far. */ private volatile T[] results; /** The function that is attached to all futures in the conjunction. Once a future @@ -215,7 +318,7 @@ final void handleCompletedFuture(T value, Throwable throwable) { @SuppressWarnings("unchecked") ResultConjunctFuture(int numTotal) { this.numTotal = numTotal; - results = (T[])new Object[numTotal]; + results = (T[]) new Object[numTotal]; } @Override @@ -235,13 +338,13 @@ public int getNumFuturesCompleted() { */ private static final class WaitingConjunctFuture extends ConjunctFuture { - /** Number of completed futures */ + /** Number of completed futures. */ private final AtomicInteger numCompleted = new AtomicInteger(0); - /** Total number of futures to wait on */ + /** Total number of futures to wait on. */ private final int numTotal; - /** Method which increments the atomic completion counter and completes or fails the WaitingFutureImpl */ + /** Method which increments the atomic completion counter and completes or fails the WaitingFutureImpl. */ private void handleCompletedFuture(Object ignored, Throwable throwable) { if (throwable == null) { if (numTotal == numCompleted.incrementAndGet()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ConjunctFutureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ConjunctFutureTest.java new file mode 100644 index 0000000000000..f92504ef84977 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/ConjunctFutureTest.java @@ -0,0 +1,246 @@ +/* + * 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.concurrent; + +import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture; +import org.apache.flink.util.TestLogger; + +import org.hamcrest.collection.IsIterableContainingInAnyOrder; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Tests for the {@link ConjunctFuture} and {@link FutureUtils.WaitingConjunctFuture}. + */ +@RunWith(Parameterized.class) +public class ConjunctFutureTest extends TestLogger { + + @Parameterized.Parameters + public static Collection parameters (){ + return Arrays.asList(new ConjunctFutureFactory(), new WaitingFutureFactory()); + } + + @Parameterized.Parameter + public FutureFactory futureFactory; + + @Test + public void testConjunctFutureFailsOnEmptyAndNull() throws Exception { + try { + futureFactory.createFuture(null); + fail(); + } catch (NullPointerException ignored) {} + + try { + futureFactory.createFuture(Arrays.asList( + new CompletableFuture<>(), + null, + new CompletableFuture<>())); + fail(); + } catch (NullPointerException ignored) {} + } + + @Test + public void testConjunctFutureCompletion() throws Exception { + // some futures that we combine + java.util.concurrent.CompletableFuture future1 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future2 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future4 = new java.util.concurrent.CompletableFuture<>(); + + // some future is initially completed + future2.complete(new Object()); + + // build the conjunct future + ConjunctFuture result = futureFactory.createFuture(Arrays.asList(future1, future2, future3, future4)); + + CompletableFuture resultMapped = result.thenAccept(value -> {}); + + assertEquals(4, result.getNumFuturesTotal()); + assertEquals(1, result.getNumFuturesCompleted()); + assertFalse(result.isDone()); + assertFalse(resultMapped.isDone()); + + // complete two more futures + future4.complete(new Object()); + assertEquals(2, result.getNumFuturesCompleted()); + assertFalse(result.isDone()); + assertFalse(resultMapped.isDone()); + + future1.complete(new Object()); + assertEquals(3, result.getNumFuturesCompleted()); + assertFalse(result.isDone()); + assertFalse(resultMapped.isDone()); + + // complete one future again + future1.complete(new Object()); + assertEquals(3, result.getNumFuturesCompleted()); + assertFalse(result.isDone()); + assertFalse(resultMapped.isDone()); + + // complete the final future + future3.complete(new Object()); + assertEquals(4, result.getNumFuturesCompleted()); + assertTrue(result.isDone()); + assertTrue(resultMapped.isDone()); + } + + @Test + public void testConjunctFutureFailureOnFirst() throws Exception { + + java.util.concurrent.CompletableFuture future1 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future2 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future4 = new java.util.concurrent.CompletableFuture<>(); + + // build the conjunct future + ConjunctFuture result = futureFactory.createFuture(Arrays.asList(future1, future2, future3, future4)); + + CompletableFuture resultMapped = result.thenAccept(value -> {}); + + assertEquals(4, result.getNumFuturesTotal()); + assertEquals(0, result.getNumFuturesCompleted()); + assertFalse(result.isDone()); + assertFalse(resultMapped.isDone()); + + future2.completeExceptionally(new IOException()); + + assertEquals(0, result.getNumFuturesCompleted()); + assertTrue(result.isDone()); + assertTrue(resultMapped.isDone()); + + try { + result.get(); + fail(); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof IOException); + } + + try { + resultMapped.get(); + fail(); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof IOException); + } + } + + @Test + public void testConjunctFutureFailureOnSuccessive() throws Exception { + + java.util.concurrent.CompletableFuture future1 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future2 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); + java.util.concurrent.CompletableFuture future4 = new java.util.concurrent.CompletableFuture<>(); + + // build the conjunct future + ConjunctFuture result = futureFactory.createFuture(Arrays.asList(future1, future2, future3, future4)); + assertEquals(4, result.getNumFuturesTotal()); + + java.util.concurrent.CompletableFuture resultMapped = result.thenAccept(value -> {}); + + future1.complete(new Object()); + future3.complete(new Object()); + future4.complete(new Object()); + + future2.completeExceptionally(new IOException()); + + assertEquals(3, result.getNumFuturesCompleted()); + assertTrue(result.isDone()); + assertTrue(resultMapped.isDone()); + + try { + result.get(); + fail(); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof IOException); + } + + try { + resultMapped.get(); + fail(); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof IOException); + } + } + + /** + * Tests that the conjunct future returns upon completion the collection of all future values + */ + @Test + public void testConjunctFutureValue() throws ExecutionException, InterruptedException { + java.util.concurrent.CompletableFuture future1 = java.util.concurrent.CompletableFuture.completedFuture(1); + java.util.concurrent.CompletableFuture future2 = java.util.concurrent.CompletableFuture.completedFuture(2L); + java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); + + ConjunctFuture> result = FutureUtils.combineAll(Arrays.asList(future1, future2, future3)); + + assertFalse(result.isDone()); + + future3.complete(.1); + + assertTrue(result.isDone()); + + assertThat(result.get(), IsIterableContainingInAnyOrder.containsInAnyOrder(1, 2L, .1)); + } + + @Test + public void testConjunctOfNone() throws Exception { + final ConjunctFuture result = futureFactory.createFuture(Collections.>emptyList()); + + assertEquals(0, result.getNumFuturesTotal()); + assertEquals(0, result.getNumFuturesCompleted()); + assertTrue(result.isDone()); + } + + /** + * Factory to create {@link ConjunctFuture} for testing. + */ + private interface FutureFactory { + ConjunctFuture createFuture(Collection> futures); + } + + private static class ConjunctFutureFactory implements FutureFactory { + + @Override + public ConjunctFuture createFuture(Collection> futures) { + return FutureUtils.combineAll(futures); + } + } + + private static class WaitingFutureFactory implements FutureFactory { + + @Override + public ConjunctFuture createFuture(Collection> futures) { + return FutureUtils.waitForAll(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 cc95e7ad38ca9..c624ef2ec8fb8 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 @@ -18,225 +18,208 @@ package org.apache.flink.runtime.concurrent; -import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture; - +import org.apache.flink.api.common.time.Time; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.testingUtils.TestingUtils; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; -import org.hamcrest.collection.IsIterableContainingInAnyOrder; + import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.mockito.invocation.InvocationOnMock; -import java.io.IOException; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; - -import static org.junit.Assert.*; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; /** - * Tests for the utility methods in {@link FutureUtils} + * Tests for the utility methods in {@link FutureUtils}. */ -@RunWith(Parameterized.class) -public class FutureUtilsTest extends TestLogger{ - - @Parameterized.Parameters - public static Collection parameters (){ - return Arrays.asList(new ConjunctFutureFactory(), new WaitingFutureFactory()); - } - - @Parameterized.Parameter - public FutureFactory futureFactory; - - @Test - public void testConjunctFutureFailsOnEmptyAndNull() throws Exception { - try { - futureFactory.createFuture(null); - fail(); - } catch (NullPointerException ignored) {} - - try { - futureFactory.createFuture(Arrays.asList( - new CompletableFuture<>(), - null, - new CompletableFuture<>())); - fail(); - } catch (NullPointerException ignored) {} - } +public class FutureUtilsTest extends TestLogger { + /** + * Tests that we can retry an operation. + */ @Test - public void testConjunctFutureCompletion() throws Exception { - // some futures that we combine - java.util.concurrent.CompletableFuture future1 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future2 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future4 = new java.util.concurrent.CompletableFuture<>(); - - // some future is initially completed - future2.complete(new Object()); - - // build the conjunct future - ConjunctFuture result = futureFactory.createFuture(Arrays.asList(future1, future2, future3, future4)); - - CompletableFuture resultMapped = result.thenAccept(value -> {}); - - assertEquals(4, result.getNumFuturesTotal()); - assertEquals(1, result.getNumFuturesCompleted()); - assertFalse(result.isDone()); - assertFalse(resultMapped.isDone()); - - // complete two more futures - future4.complete(new Object()); - assertEquals(2, result.getNumFuturesCompleted()); - assertFalse(result.isDone()); - assertFalse(resultMapped.isDone()); - - future1.complete(new Object()); - assertEquals(3, result.getNumFuturesCompleted()); - assertFalse(result.isDone()); - assertFalse(resultMapped.isDone()); - - // complete one future again - future1.complete(new Object()); - assertEquals(3, result.getNumFuturesCompleted()); - assertFalse(result.isDone()); - assertFalse(resultMapped.isDone()); - - // complete the final future - future3.complete(new Object()); - assertEquals(4, result.getNumFuturesCompleted()); - assertTrue(result.isDone()); - assertTrue(resultMapped.isDone()); + public void testRetrySuccess() throws Exception { + final int retries = 10; + final AtomicInteger atomicInteger = new AtomicInteger(0); + CompletableFuture retryFuture = FutureUtils.retry( + () -> + CompletableFuture.supplyAsync( + () -> { + if (atomicInteger.incrementAndGet() == retries) { + return true; + } else { + throw new FlinkFutureException("Test exception"); + } + }, + TestingUtils.defaultExecutor()), + retries, + TestingUtils.defaultExecutor()); + + assertTrue(retryFuture.get()); + assertTrue(retries == atomicInteger.get()); } - @Test - public void testConjunctFutureFailureOnFirst() throws Exception { - - java.util.concurrent.CompletableFuture future1 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future2 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future4 = new java.util.concurrent.CompletableFuture<>(); - - // build the conjunct future - ConjunctFuture result = futureFactory.createFuture(Arrays.asList(future1, future2, future3, future4)); - - CompletableFuture resultMapped = result.thenAccept(value -> {}); - - assertEquals(4, result.getNumFuturesTotal()); - assertEquals(0, result.getNumFuturesCompleted()); - assertFalse(result.isDone()); - assertFalse(resultMapped.isDone()); - - future2.completeExceptionally(new IOException()); - - assertEquals(0, result.getNumFuturesCompleted()); - assertTrue(result.isDone()); - assertTrue(resultMapped.isDone()); + /** + * Tests that a retry future is failed after all retries have been consumed. + */ + @Test(expected = FutureUtils.RetryException.class) + public void testRetryFailure() throws Throwable { + final int retries = 3; - try { - result.get(); - fail(); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof IOException); - } + CompletableFuture retryFuture = FutureUtils.retry( + () -> FutureUtils.completedExceptionally(new FlinkException("Test exception")), + retries, + TestingUtils.defaultExecutor()); try { - resultMapped.get(); - fail(); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof IOException); + retryFuture.get(); + } catch (ExecutionException ee) { + throw ExceptionUtils.stripExecutionException(ee); } } + /** + * Tests that we can cancel a retry future. + */ @Test - public void testConjunctFutureFailureOnSuccessive() throws Exception { - - java.util.concurrent.CompletableFuture future1 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future2 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); - java.util.concurrent.CompletableFuture future4 = new java.util.concurrent.CompletableFuture<>(); - - // build the conjunct future - ConjunctFuture result = futureFactory.createFuture(Arrays.asList(future1, future2, future3, future4)); - assertEquals(4, result.getNumFuturesTotal()); - - java.util.concurrent.CompletableFuture resultMapped = result.thenAccept(value -> {}); - - future1.complete(new Object()); - future3.complete(new Object()); - future4.complete(new Object()); - - future2.completeExceptionally(new IOException()); - - assertEquals(3, result.getNumFuturesCompleted()); - assertTrue(result.isDone()); - assertTrue(resultMapped.isDone()); - - try { - result.get(); - fail(); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof IOException); - } - - try { - resultMapped.get(); - fail(); - } catch (ExecutionException e) { - assertTrue(e.getCause() instanceof IOException); + public void testRetryCancellation() throws Exception { + final int retries = 10; + final AtomicInteger atomicInteger = new AtomicInteger(0); + final OneShotLatch notificationLatch = new OneShotLatch(); + final OneShotLatch waitLatch = new OneShotLatch(); + final AtomicReference atomicThrowable = new AtomicReference<>(null); + + CompletableFuture retryFuture = FutureUtils.retry( + () -> + CompletableFuture.supplyAsync( + () -> { + if (atomicInteger.incrementAndGet() == 2) { + notificationLatch.trigger(); + try { + waitLatch.await(); + } catch (InterruptedException e) { + atomicThrowable.compareAndSet(null, e); + } + } + + throw new FlinkFutureException("Test exception"); + }, + TestingUtils.defaultExecutor()), + retries, + TestingUtils.defaultExecutor()); + + // await that we have failed once + notificationLatch.await(); + + assertFalse(retryFuture.isDone()); + + // cancel the retry future + retryFuture.cancel(false); + + // let the retry operation continue + waitLatch.trigger(); + + assertTrue(retryFuture.isCancelled()); + assertEquals(2, atomicInteger.get()); + + if (atomicThrowable.get() != null) { + throw new FlinkException("Exception occurred in the retry operation.", atomicThrowable.get()); } } /** - * Tests that the conjunct future returns upon completion the collection of all future values + * Tests that retry with delay fails after having exceeded all retries. */ - @Test - public void testConjunctFutureValue() throws ExecutionException, InterruptedException { - java.util.concurrent.CompletableFuture future1 = java.util.concurrent.CompletableFuture.completedFuture(1); - java.util.concurrent.CompletableFuture future2 = java.util.concurrent.CompletableFuture.completedFuture(2L); - java.util.concurrent.CompletableFuture future3 = new java.util.concurrent.CompletableFuture<>(); - - ConjunctFuture> result = FutureUtils.combineAll(Arrays.asList(future1, future2, future3)); - - assertFalse(result.isDone()); - - future3.complete(.1); - - assertTrue(result.isDone()); + @Test(expected = FutureUtils.RetryException.class) + public void testRetryWithDelayFailure() throws Throwable { + CompletableFuture retryFuture = FutureUtils.retryWithDelay( + () -> FutureUtils.completedExceptionally(new FlinkException("Test exception")), + 3, + Time.milliseconds(1L), + TestingUtils.defaultScheduledExecutor()); - assertThat(result.get(), IsIterableContainingInAnyOrder.containsInAnyOrder(1, 2L, .1)); + try { + retryFuture.get(TestingUtils.TIMEOUT().toMilliseconds(), TimeUnit.MILLISECONDS); + } catch (ExecutionException ee) { + throw ExceptionUtils.stripExecutionException(ee); + } } + /** + * Tests that the delay is respected between subsequent retries of a retry future with retry delay. + */ @Test - public void testConjunctOfNone() throws Exception { - final ConjunctFuture result = futureFactory.createFuture(Collections.>emptyList()); - - assertEquals(0, result.getNumFuturesTotal()); - assertEquals(0, result.getNumFuturesCompleted()); - assertTrue(result.isDone()); + public void testRetryWithDelay() throws Exception { + final int retries = 4; + final Time delay = Time.milliseconds(50L); + final AtomicInteger countDown = new AtomicInteger(retries); + + CompletableFuture retryFuture = FutureUtils.retryWithDelay( + () -> { + if (countDown.getAndDecrement() == 0) { + return CompletableFuture.completedFuture(true); + } else { + return FutureUtils.completedExceptionally(new FlinkException("Test exception.")); + } + }, + retries, + delay, + TestingUtils.defaultScheduledExecutor()); + + long start = System.currentTimeMillis(); + + Boolean result = retryFuture.get(); + + long completionTime = System.currentTimeMillis() - start; + + assertTrue(result); + assertTrue("The completion time should be at least rertries times delay between retries.", completionTime >= retries * delay.toMilliseconds()); } /** - * Factory to create {@link ConjunctFuture} for testing. + * Tests that all scheduled tasks are canceled if the retry future is being cancelled. */ - private interface FutureFactory { - ConjunctFuture createFuture(Collection> futures); - } - - private static class ConjunctFutureFactory implements FutureFactory { - - @Override - public ConjunctFuture createFuture(Collection> futures) { - return FutureUtils.combineAll(futures); - } - } - - private static class WaitingFutureFactory implements FutureFactory { - - @Override - public ConjunctFuture createFuture(Collection> futures) { - return FutureUtils.waitForAll(futures); - } + @Test + public void testRetryWithDelayCancellation() { + ScheduledFuture scheduledFutureMock = mock(ScheduledFuture.class); + ScheduledExecutor scheduledExecutorMock = mock(ScheduledExecutor.class); + doReturn(scheduledFutureMock).when(scheduledExecutorMock).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + doAnswer( + (InvocationOnMock invocation) -> { + invocation.getArgumentAt(0, Runnable.class).run(); + return null; + }).when(scheduledExecutorMock).execute(any(Runnable.class)); + + CompletableFuture retryFuture = FutureUtils.retryWithDelay( + () -> FutureUtils.completedExceptionally(new FlinkException("Test exception")), + 1, + TestingUtils.infiniteTime(), + scheduledExecutorMock); + + assertFalse(retryFuture.isDone()); + + verify(scheduledExecutorMock).schedule(any(Runnable.class), anyLong(), any(TimeUnit.class)); + + retryFuture.cancel(false); + + assertTrue(retryFuture.isCancelled()); + verify(scheduledFutureMock).cancel(anyBoolean()); } } From 8119baaf1bd5f22a6fd14909b61f01c644b62a66 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Wed, 6 Sep 2017 11:15:38 +0200 Subject: [PATCH 102/129] [FLINK-7430] Set StreamTask.isRunning to false after closing StreamOperators Closing StreamOperators is still part of the StreamTask's running lifecycle, because operators which perform asynchronous operations usually finish their work when the StreamOperator is closed. Since this also entails that errors can occur and that a checkpointing operation is triggered, we should only set the StreamTask's isRunning to false after all StreamOperators have been closed. Furthermore, this commit introduces a while guard for the waiting condition in ContinuousFileReaderOperator#close. This closes #4650. --- .../source/ContinuousFileReaderOperator.java | 5 +- .../streaming/runtime/tasks/StreamTask.java | 6 +- .../runtime/tasks/StreamTaskTest.java | 91 +++++++++++++++++++ 3 files changed, 99 insertions(+), 3 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java index e14cfda3b4608..78e181a978b3c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileReaderOperator.java @@ -196,11 +196,14 @@ public void dispose() throws Exception { public void close() throws Exception { super.close(); + // make sure that we hold the checkpointing lock + Thread.holdsLock(checkpointLock); + // close the reader to signal that no more splits will come. By doing this, // the reader will exit as soon as it finishes processing the already pending splits. // This method will wait until then. Further cleaning up is handled by the dispose(). - if (reader != null && reader.isAlive() && reader.isRunning()) { + while (reader != null && reader.isAlive() && reader.isRunning()) { reader.close(); checkpointLock.wait(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java index 310df4de6c141..6089240c8db51 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java @@ -277,10 +277,12 @@ public final void invoke() throws Exception { // we also need to make sure that no triggers fire concurrently with the close logic // at the same time, this makes sure that during any "regular" exit where still synchronized (lock) { - isRunning = false; - // this is part of the main logic, so if this fails, the task is considered failed closeAllOperators(); + + // only set the StreamTask to not running after all operators have been closed! + // See FLINK-7430 + isRunning = false; } LOG.debug("Closed operators for task {}", getName()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index a2dc6c4f2fa0d..9bb91ad7ba8a6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -56,6 +56,8 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; @@ -73,6 +75,7 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.taskmanager.TaskExecutionStateListener; import org.apache.flink.runtime.taskmanager.TaskManagerActions; +import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.DirectExecutorService; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.streaming.api.TimeCharacteristic; @@ -111,12 +114,14 @@ import java.util.Comparator; import java.util.List; import java.util.PriorityQueue; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.RunnableFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import scala.concurrent.Await; import scala.concurrent.Future; @@ -717,10 +722,96 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { Assert.assertNull(checkpointResult.get(0)); } + /** + * Tests that the StreamTask first closes alls its operators before setting its + * state to not running (isRunning == false) + * + *

See FLINK-7430. + */ + @Test + public void testOperatorClosingBeforeStopRunning() throws Throwable { + Configuration taskConfiguration = new Configuration(); + StreamConfig streamConfig = new StreamConfig(taskConfiguration); + streamConfig.setStreamOperator(new BlockingCloseStreamOperator()); + streamConfig.setOperatorID(new OperatorID()); + + MockEnvironment mockEnvironment = new MockEnvironment( + "Test Task", + 32L * 1024L, + new MockInputSplitProvider(), + 1, + taskConfiguration, + new ExecutionConfig()); + StreamTask streamTask = new NoOpStreamTask<>(mockEnvironment); + final AtomicReference atomicThrowable = new AtomicReference<>(null); + + CompletableFuture invokeFuture = CompletableFuture.runAsync( + () -> { + try { + streamTask.invoke(); + } catch (Exception e) { + atomicThrowable.set(e); + } + }, + TestingUtils.defaultExecutor()); + + BlockingCloseStreamOperator.IN_CLOSE.await(); + + // check that the StreamTask is not yet in isRunning == false + assertTrue(streamTask.isRunning()); + + // let the operator finish its close operation + BlockingCloseStreamOperator.FINISH_CLOSE.trigger(); + + // wait until the invoke is complete + invokeFuture.get(); + + // now the StreamTask should no longer be running + assertFalse(streamTask.isRunning()); + + // check if an exception occurred + if (atomicThrowable.get() != null) { + throw atomicThrowable.get(); + } + } + // ------------------------------------------------------------------------ // Test Utilities // ------------------------------------------------------------------------ + private static class NoOpStreamTask> extends StreamTask { + + public NoOpStreamTask(Environment environment) { + setEnvironment(environment); + } + + @Override + protected void init() throws Exception {} + + @Override + protected void run() throws Exception {} + + @Override + protected void cleanup() throws Exception {} + + @Override + protected void cancelTask() throws Exception {} + } + + private static class BlockingCloseStreamOperator extends AbstractStreamOperator { + private static final long serialVersionUID = -9042150529568008847L; + + public static final OneShotLatch IN_CLOSE = new OneShotLatch(); + public static final OneShotLatch FINISH_CLOSE = new OneShotLatch(); + + @Override + public void close() throws Exception { + IN_CLOSE.trigger(); + FINISH_CLOSE.await(); + super.close(); + } + } + private static class TestingExecutionStateListener implements TaskExecutionStateListener { private ExecutionState executionState = null; From d0636c8ce01d3d0cf032235647887c3560c4833a Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Thu, 31 Aug 2017 17:10:36 +0200 Subject: [PATCH 103/129] [hotfix][kafka][docs] Add warning regarding data losses when writing to Kafka --- docs/dev/connectors/kafka.md | 23 +++++++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/docs/dev/connectors/kafka.md b/docs/dev/connectors/kafka.md index 042ad11bc7705..f95c8c09ee477 100644 --- a/docs/dev/connectors/kafka.md +++ b/docs/dev/connectors/kafka.md @@ -475,8 +475,14 @@ are other constructor variants that allow providing the following: ### Kafka Producers and Fault Tolerance -With Flink's checkpointing enabled, the Flink Kafka Producer can provide -at-least-once delivery guarantees. +#### Kafka 0.8 + +Before 0.9 Kafka did not provide any mechanisms to guarantee at-least-once or exactly-once semantics. + +#### Kafka 0.9 and 0.10 + +With Flink's checkpointing enabled, the `FlinkKafkaProducer09` and `FlinkKafkaProducer010` +can provide at-least-once delivery guarantees. Besides enabling Flink's checkpointing, you should also configure the setter methods `setLogFailuresOnly(boolean)` and `setFlushOnCheckpoint(boolean)` appropriately, @@ -499,6 +505,19 @@ we recommend setting the number of retries to a higher value. **Note**: There is currently no transactional producer for Kafka, so Flink can not guarantee exactly-once delivery into a Kafka topic. +

+ Attention: Depending on your Kafka configuration, even after Kafka acknowledges + writes you can still experience data loss. In particular keep in mind the following Kafka settings: +
    +
  • acks
  • +
  • log.flush.interval.messages
  • +
  • log.flush.interval.ms
  • +
  • log.flush.*
  • +
+ Default values for the above options can easily lead to data loss. Please refer to Kafka documentation + for more explanation. +
+ ## Using Kafka timestamps and Flink event time in Kafka 0.10 Since Apache Kafka 0.10+, Kafka's messages can carry [timestamps](https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message), indicating From 93369e79eb21f17791ddad4e03a18980be6eabfb Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 11 Aug 2017 15:53:46 +0800 Subject: [PATCH 104/129] [FLINK-7407] [kafka] Adapt AbstractPartitionDiscoverer to handle non-contiguous partition metadata Previously, the AbstractPartitionDiscoverer tracked discovered partitions by keeping only the largest discovered partition id. All fetched partition metadata with ids smaller than this id would be considered as discovered. This assumption of contiguous partition ids is too naive for corner cases where there may be undiscovered partitions that were temporariliy unavilable before and were shadowed by discoverered partitions with largerer partition ids. This commit changes to use a set to track seen partitions. This also removes the need of pre-sorting fetched partitions. --- .../AbstractPartitionDiscoverer.java | 17 ++++----- .../kafka/internals/KafkaTopicPartition.java | 5 --- .../AbstractPartitionDiscovererTest.java | 35 +++++++++++++++++++ 3 files changed, 41 insertions(+), 16 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java index 39645be129f3d..8240f607009a5 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java @@ -17,10 +17,10 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Map; +import java.util.Set; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -68,7 +68,7 @@ public abstract class AbstractPartitionDiscoverer { * to keep track of only the largest partition id because Kafka partition numbers are only * allowed to be increased and has incremental ids. */ - private Map topicsToLargestDiscoveredPartitionId; + private Set discoveredPartitions; public AbstractPartitionDiscoverer( KafkaTopicsDescriptor topicsDescriptor, @@ -78,7 +78,7 @@ public AbstractPartitionDiscoverer( this.topicsDescriptor = checkNotNull(topicsDescriptor); this.indexOfThisSubtask = indexOfThisSubtask; this.numParallelSubtasks = numParallelSubtasks; - this.topicsToLargestDiscoveredPartitionId = new HashMap<>(); + this.discoveredPartitions = new HashSet<>(); } /** @@ -149,10 +149,6 @@ public List discoverPartitions() throws WakeupException, Cl if (newDiscoveredPartitions == null || newDiscoveredPartitions.isEmpty()) { throw new RuntimeException("Unable to retrieve any partitions with KafkaTopicsDescriptor: " + topicsDescriptor); } else { - // sort so that we make sure the topicsToLargestDiscoveredPartitionId state is updated - // with incremental partition ids of the same topics (otherwise some partition ids may be skipped) - KafkaTopicPartition.sort(newDiscoveredPartitions); - Iterator iter = newDiscoveredPartitions.iterator(); KafkaTopicPartition nextPartition; while (iter.hasNext()) { @@ -196,7 +192,7 @@ public List discoverPartitions() throws WakeupException, Cl */ public boolean setAndCheckDiscoveredPartition(KafkaTopicPartition partition) { if (isUndiscoveredPartition(partition)) { - topicsToLargestDiscoveredPartitionId.put(partition.getTopic(), partition.getPartition()); + discoveredPartitions.add(partition); return KafkaTopicPartitionAssigner.assign(partition, numParallelSubtasks) == indexOfThisSubtask; } @@ -246,8 +242,7 @@ public static final class ClosedException extends Exception { } private boolean isUndiscoveredPartition(KafkaTopicPartition partition) { - return !topicsToLargestDiscoveredPartitionId.containsKey(partition.getTopic()) - || partition.getPartition() > topicsToLargestDiscoveredPartitionId.get(partition.getTopic()); + return !discoveredPartitions.contains(partition); } public static boolean shouldAssignToThisSubtask(KafkaTopicPartition partition, int indexOfThisSubtask, int numParallelSubtasks) { diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java index 3500cd81edc3f..d35d5856f15ef 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartition.java @@ -19,7 +19,6 @@ import java.io.Serializable; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -131,8 +130,4 @@ public int compare(KafkaTopicPartition p1, KafkaTopicPartition p2) { } } } - - public static void sort(List partitions) { - Collections.sort(partitions, new Comparator()); - } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java index 2633b951ae1f8..e9f1537ed1be1 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java @@ -394,6 +394,41 @@ public void testDeterministicAssignmentWithDifferentFetchedPartitionOrdering() t } } + @Test + public void testNonContiguousPartitionIdDiscovery() throws Exception { + List mockGetAllPartitionsForTopicsReturn1 = Arrays.asList( + new KafkaTopicPartition("test-topic", 1), + new KafkaTopicPartition("test-topic", 4)); + + List mockGetAllPartitionsForTopicsReturn2 = Arrays.asList( + new KafkaTopicPartition("test-topic", 0), + new KafkaTopicPartition("test-topic", 1), + new KafkaTopicPartition("test-topic", 2), + new KafkaTopicPartition("test-topic", 3), + new KafkaTopicPartition("test-topic", 4)); + + TestPartitionDiscoverer partitionDiscoverer = new TestPartitionDiscoverer( + topicsDescriptor, + 0, + 1, + TestPartitionDiscoverer.createMockGetAllTopicsSequenceFromFixedReturn(Collections.singletonList("test-topic")), + // first metadata fetch has missing partitions that appears only in the second fetch; + // need to create new modifiable lists for each fetch, since internally Iterable.remove() is used. + Arrays.asList(new ArrayList<>(mockGetAllPartitionsForTopicsReturn1), new ArrayList<>(mockGetAllPartitionsForTopicsReturn2))); + partitionDiscoverer.open(); + + List discoveredPartitions1 = partitionDiscoverer.discoverPartitions(); + assertEquals(2, discoveredPartitions1.size()); + assertTrue(discoveredPartitions1.contains(new KafkaTopicPartition("test-topic", 1))); + assertTrue(discoveredPartitions1.contains(new KafkaTopicPartition("test-topic", 4))); + + List discoveredPartitions2 = partitionDiscoverer.discoverPartitions(); + assertEquals(3, discoveredPartitions2.size()); + assertTrue(discoveredPartitions2.contains(new KafkaTopicPartition("test-topic", 0))); + assertTrue(discoveredPartitions2.contains(new KafkaTopicPartition("test-topic", 2))); + assertTrue(discoveredPartitions2.contains(new KafkaTopicPartition("test-topic", 3))); + } + private boolean contains(List partitions, int partition) { for (KafkaTopicPartition ktp : partitions) { if (ktp.getPartition() == partition) { From 36412c6f640f1b20070461f3b25f1453c93d194d Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 11 Aug 2017 15:57:52 +0800 Subject: [PATCH 105/129] [hotfix] [kafka] Remove unused shouldAssignToThisSubtask method in AbstractPartitionDiscoverer This closes #4526. --- .../kafka/internals/AbstractPartitionDiscoverer.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java index 8240f607009a5..b336fdc57061b 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscoverer.java @@ -244,8 +244,4 @@ public static final class ClosedException extends Exception { private boolean isUndiscoveredPartition(KafkaTopicPartition partition) { return !discoveredPartitions.contains(partition); } - - public static boolean shouldAssignToThisSubtask(KafkaTopicPartition partition, int indexOfThisSubtask, int numParallelSubtasks) { - return Math.abs(partition.hashCode() % numParallelSubtasks) == indexOfThisSubtask; - } } From eaafb61f8f4e69ec683f8e4cab5c1c6f34a9b4ff Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 14 Aug 2017 15:16:54 +0800 Subject: [PATCH 106/129] [FLINK-7440] [kinesis] Eagerly check serializability of deserialization schema in FlinkKinesisConsumer This commit also adds tests for verifying that the FlinkKinesisConsumer itself is serializable. --- .../kinesis/FlinkKinesisConsumer.java | 8 +- .../kinesis/FlinkKinesisConsumerTest.java | 74 +++++++++++++++++++ 2 files changed, 81 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java index 5689229f0604a..a3681eca52fee 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumer.java @@ -42,6 +42,7 @@ import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisDeserializationSchemaWrapper; import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil; import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.util.InstantiationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -173,7 +174,12 @@ public FlinkKinesisConsumer(List streams, KinesisDeserializationSchema("test-stream", new NonSerializableDeserializationSchema(), testConfig); + } + + @Test + public void testCreateWithSerializableDeserializer() { + Properties testConfig = new Properties(); + testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); + testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); + + new FlinkKinesisConsumer<>("test-stream", new SerializableDeserializationSchema(), testConfig); + } + + @Test + public void testConsumerIsSerializable() { + Properties testConfig = new Properties(); + testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); + testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); + + FlinkKinesisConsumer consumer = new FlinkKinesisConsumer<>("test-stream", new SimpleStringSchema(), testConfig); + assertTrue(InstantiationUtil.isSerializable(consumer)); + } + // ---------------------------------------------------------------------- // Tests related to state initialization // ---------------------------------------------------------------------- @@ -1030,4 +1073,35 @@ private HashMap getFakeRestoredStore(String s return fakeRestoredState; } + + /** + * A non-serializable {@link KinesisDeserializationSchema} (because it is a nested class with reference + * to the enclosing class, which is not serializable) used for testing. + */ + private final class NonSerializableDeserializationSchema implements KinesisDeserializationSchema { + @Override + public String deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId) throws IOException { + return new String(recordValue); + } + + @Override + public TypeInformation getProducedType() { + return BasicTypeInfo.STRING_TYPE_INFO; + } + } + + /** + * A static, serializable {@link KinesisDeserializationSchema}. + */ + private static final class SerializableDeserializationSchema implements KinesisDeserializationSchema { + @Override + public String deserialize(byte[] recordValue, String partitionKey, String seqNum, long approxArrivalTimestamp, String stream, String shardId) throws IOException { + return new String(recordValue); + } + + @Override + public TypeInformation getProducedType() { + return BasicTypeInfo.STRING_TYPE_INFO; + } + } } From 98737f9a875f1899cb14b3dcef1bd2ac1c6530ba Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 14 Aug 2017 15:35:43 +0800 Subject: [PATCH 107/129] [FLINK-7440] [kinesis] Eagerly check that provided schema and partitioner is serializable in FlinkKinesisProducer This commit also adds a test to verify that the FlinkKinesisProducer is serializable. This closes #4537. --- .../kinesis/FlinkKinesisProducer.java | 19 ++- .../kinesis/FlinkKinesisProducerTest.java | 161 ++++++++++++++++++ 2 files changed, 175 insertions(+), 5 deletions(-) create mode 100644 flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java index 04d7055ca8230..e0d3e38e97226 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java @@ -17,7 +17,6 @@ package org.apache.flink.streaming.connectors.kinesis; -import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; @@ -25,6 +24,7 @@ import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil; import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.PropertiesUtil; import com.amazonaws.services.kinesis.producer.Attempt; @@ -35,14 +35,15 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; import java.util.List; -import java.util.Objects; import java.util.Properties; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -123,7 +124,11 @@ public FlinkKinesisProducer(KinesisSerializationSchema schema, Properties c // check the configuration properties for any conflicting settings KinesisConfigUtil.validateProducerConfiguration(this.configProps); - ClosureCleaner.ensureSerializable(Objects.requireNonNull(schema)); + checkNotNull(schema, "serialization schema cannot be null"); + checkArgument( + InstantiationUtil.isSerializable(schema), + "The provided serialization schema is not serializable: " + schema.getClass().getName() + ". " + + "Please check that it does not contain references to non-serializable instances."); this.schema = schema; } @@ -154,8 +159,12 @@ public void setDefaultPartition(String defaultPartition) { } public void setCustomPartitioner(KinesisPartitioner partitioner) { - Objects.requireNonNull(partitioner); - ClosureCleaner.ensureSerializable(partitioner); + checkNotNull(partitioner, "partitioner cannot be null"); + checkArgument( + InstantiationUtil.isSerializable(partitioner), + "The provided custom partitioner is not serializable: " + partitioner.getClass().getName() + ". " + + "Please check that it does not contain references to non-serializable instances."); + this.customPartitioner = partitioner; } diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java new file mode 100644 index 0000000000000..ac03cfed0c898 --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducerTest.java @@ -0,0 +1,161 @@ +/* + * 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.kinesis; + +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema; +import org.apache.flink.streaming.util.serialization.SimpleStringSchema; +import org.apache.flink.util.InstantiationUtil; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.nio.ByteBuffer; +import java.util.Properties; + +import static org.junit.Assert.assertTrue; + +/** + * Suite of {@link FlinkKinesisProducer} tests. + */ +public class FlinkKinesisProducerTest { + + @Rule + public ExpectedException exception = ExpectedException.none(); + + // ---------------------------------------------------------------------- + // Tests to verify serializability + // ---------------------------------------------------------------------- + + @Test + public void testCreateWithNonSerializableDeserializerFails() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("The provided serialization schema is not serializable"); + + Properties testConfig = new Properties(); + testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); + testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); + + new FlinkKinesisProducer<>(new NonSerializableSerializationSchema(), testConfig); + } + + @Test + public void testCreateWithSerializableDeserializer() { + Properties testConfig = new Properties(); + testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); + testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); + + new FlinkKinesisProducer<>(new SerializableSerializationSchema(), testConfig); + } + + @Test + public void testConfigureWithNonSerializableCustomPartitionerFails() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("The provided custom partitioner is not serializable"); + + Properties testConfig = new Properties(); + testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); + testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); + + new FlinkKinesisProducer<>(new SimpleStringSchema(), testConfig) + .setCustomPartitioner(new NonSerializableCustomPartitioner()); + } + + @Test + public void testConfigureWithSerializableCustomPartitioner() { + Properties testConfig = new Properties(); + testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); + testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); + + new FlinkKinesisProducer<>(new SimpleStringSchema(), testConfig) + .setCustomPartitioner(new SerializableCustomPartitioner()); + } + + @Test + public void testConsumerIsSerializable() { + Properties testConfig = new Properties(); + testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); + testConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); + + FlinkKinesisProducer consumer = new FlinkKinesisProducer<>(new SimpleStringSchema(), testConfig); + assertTrue(InstantiationUtil.isSerializable(consumer)); + } + + // ---------------------------------------------------------------------- + // Utility test classes + // ---------------------------------------------------------------------- + + /** + * A non-serializable {@link KinesisSerializationSchema} (because it is a nested class with reference + * to the enclosing class, which is not serializable) used for testing. + */ + private final class NonSerializableSerializationSchema implements KinesisSerializationSchema { + @Override + public ByteBuffer serialize(String element) { + return ByteBuffer.wrap(element.getBytes()); + } + + @Override + public String getTargetStream(String element) { + return "test-stream"; + } + } + + /** + * A static, serializable {@link KinesisSerializationSchema}. + */ + private static final class SerializableSerializationSchema implements KinesisSerializationSchema { + @Override + public ByteBuffer serialize(String element) { + return ByteBuffer.wrap(element.getBytes()); + } + + @Override + public String getTargetStream(String element) { + return "test-stream"; + } + } + + /** + * A non-serializable {@link KinesisPartitioner} (because it is a nested class with reference + * to the enclosing class, which is not serializable) used for testing. + */ + private final class NonSerializableCustomPartitioner extends KinesisPartitioner { + @Override + public String getPartitionId(String element) { + return "test-partition"; + } + } + + /** + * A static, serializable {@link KinesisPartitioner}. + */ + private static final class SerializableCustomPartitioner extends KinesisPartitioner { + @Override + public String getPartitionId(String element) { + return "test-partition"; + } + } +} From 9ed5d9a180dcd871e33bf8982434e3afd90ed295 Mon Sep 17 00:00:00 2001 From: Bowen Li Date: Thu, 3 Aug 2017 20:59:02 -0700 Subject: [PATCH 108/129] [FLINK-7367] [kinesis] Generalize configuration for FlinkKinesisProducer properties --- docs/dev/connectors/kinesis.md | 27 ++++--- .../kinesis/FlinkKinesisProducer.java | 25 ++----- .../config/ProducerConfigConstants.java | 16 ++++- .../kinesis/util/KinesisConfigUtil.java | 48 +++++++++++-- .../kinesis/FlinkKinesisConsumerTest.java | 72 ------------------- .../kinesis/util/KinesisConfigUtilTest.java | 66 +++++++++++++++++ 6 files changed, 147 insertions(+), 107 deletions(-) create mode 100644 flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java diff --git a/docs/dev/connectors/kinesis.md b/docs/dev/connectors/kinesis.md index 5fbf24b8d3e7f..1eea308fd96df 100644 --- a/docs/dev/connectors/kinesis.md +++ b/docs/dev/connectors/kinesis.md @@ -256,23 +256,29 @@ consumer when calling this API can also be modified by using the other keys pref ## Kinesis Producer -The `FlinkKinesisProducer` is used for putting data from a Flink stream into a Kinesis stream. Note that the producer is not participating in -Flink's checkpointing and doesn't provide exactly-once processing guarantees. -Also, the Kinesis producer does not guarantee that records are written in order to the shards (See [here](https://github.com/awslabs/amazon-kinesis-producer/issues/23) and [here](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax) for more details). +The `FlinkKinesisProducer` uses [Kinesis Producer Library (KPL)](http://docs.aws.amazon.com/streams/latest/dev/developing-producers-with-kpl.html) to put data from a Flink stream into a Kinesis stream. + +Note that the producer is not participating in Flink's checkpointing and doesn't provide exactly-once processing guarantees. Also, the Kinesis producer does not guarantee that records are written in order to the shards (See [here](https://github.com/awslabs/amazon-kinesis-producer/issues/23) and [here](http://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#API_PutRecord_RequestSyntax) for more details). In case of a failure or a resharding, data will be written again to Kinesis, leading to duplicates. This behavior is usually called "at-least-once" semantics. To put data into a Kinesis stream, make sure the stream is marked as "ACTIVE" in the AWS dashboard. -For the monitoring to work, the user accessing the stream needs access to the Cloud watch service. +For the monitoring to work, the user accessing the stream needs access to the CloudWatch service.
{% highlight java %} Properties producerConfig = new Properties(); +// Required configs producerConfig.put(ProducerConfigConstants.AWS_REGION, "us-east-1"); producerConfig.put(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); producerConfig.put(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); +// Optional configs +producerConfig.put("AggregationMaxCount", "4294967295"); +producerConfig.put("CollectionMaxCount", "1000"); +producerConfig.put("RecordTtl", "30000"); +producerConfig.put("RequestTimeout", "6000"); FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>(new SimpleStringSchema(), producerConfig); kinesis.setFailOnError(true); @@ -286,9 +292,15 @@ simpleStringStream.addSink(kinesis);
{% highlight scala %} val producerConfig = new Properties(); +// Required configs producerConfig.put(ProducerConfigConstants.AWS_REGION, "us-east-1"); producerConfig.put(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); producerConfig.put(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); +// Optional configs +producerConfig.put("AggregationMaxCount", "4294967295"); +producerConfig.put("CollectionMaxCount", "1000"); +producerConfig.put("RecordTtl", "30000"); +producerConfig.put("RequestTimeout", "6000"); val kinesis = new FlinkKinesisProducer[String](new SimpleStringSchema, producerConfig); kinesis.setFailOnError(true); @@ -301,15 +313,14 @@ simpleStringStream.addSink(kinesis);
-The above is a simple example of using the producer. Configuration for the producer with the mandatory configuration values is supplied with a `java.util.Properties` -instance as described above for the consumer. The example demonstrates producing a single Kinesis stream in the AWS region "us-east-1". +The above is a simple example of using the producer. To initialize `FlinkKinesisProducer`, users are required to pass in `AWS_REGION`, `AWS_ACCESS_KEY_ID`, and `AWS_SECRET_ACCESS_KEY` via a `java.util.Properties` instance. Users can also pass in KPL's configurations as optional parameters to customize the KPL underlying `FlinkKinesisProducer`. The full list of KPL configs and explanations can be found [here](https://github.com/awslabs/amazon-kinesis-producer/blob/master/java/amazon-kinesis-producer-sample/default_config.properties). The example demonstrates producing a single Kinesis stream in the AWS region "us-east-1". + +If users don't specify any KPL configs and values, `FlinkKinesisProducer` will use default config values of KPL, except `RateLimit`. `RateLimit` limits the maximum allowed put rate for a shard, as a percentage of the backend limits. KPL's default value is 150 but it makes KPL throw `RateLimitExceededException` too frequently and breaks Flink sink as a result. Thus `FlinkKinesisProducer` overrides KPL's default value to 100. Instead of a `SerializationSchema`, it also supports a `KinesisSerializationSchema`. The `KinesisSerializationSchema` allows to send the data to multiple streams. This is done using the `KinesisSerializationSchema.getTargetStream(T element)` method. Returning `null` there will instruct the producer to write the element to the default stream. Otherwise, the returned stream name is used. -Other optional configuration keys for the producer can be found in `ProducerConfigConstants`. - ## Using Non-AWS Kinesis Endpoints for Testing diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java index e0d3e38e97226..1f5e64c1fade3 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisProducer.java @@ -19,13 +19,11 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema; import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil; import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil; import org.apache.flink.streaming.util.serialization.SerializationSchema; import org.apache.flink.util.InstantiationUtil; -import org.apache.flink.util.PropertiesUtil; import com.amazonaws.services.kinesis.producer.Attempt; import com.amazonaws.services.kinesis.producer.KinesisProducer; @@ -91,7 +89,7 @@ public class FlinkKinesisProducer extends RichSinkFunction { * This is a constructor supporting Flink's {@see SerializationSchema}. * * @param schema Serialization schema for the data type - * @param configProps The properties used to configure AWS credentials and AWS region + * @param configProps The properties used to configure KinesisProducer, including AWS credentials and AWS region */ public FlinkKinesisProducer(final SerializationSchema schema, Properties configProps) { @@ -116,13 +114,11 @@ public String getTargetStream(OUT element) { * This is a constructor supporting {@see KinesisSerializationSchema}. * * @param schema Kinesis serialization schema for the data type - * @param configProps The properties used to configure AWS credentials and AWS region + * @param configProps The properties used to configure KinesisProducer, including AWS credentials and AWS region */ public FlinkKinesisProducer(KinesisSerializationSchema schema, Properties configProps) { - this.configProps = checkNotNull(configProps, "configProps can not be null"); - - // check the configuration properties for any conflicting settings - KinesisConfigUtil.validateProducerConfiguration(this.configProps); + checkNotNull(configProps, "configProps can not be null"); + this.configProps = KinesisConfigUtil.replaceDeprecatedProducerKeys(configProps); checkNotNull(schema, "serialization schema cannot be null"); checkArgument( @@ -174,18 +170,9 @@ public void setCustomPartitioner(KinesisPartitioner partitioner) { public void open(Configuration parameters) throws Exception { super.open(parameters); - KinesisProducerConfiguration producerConfig = new KinesisProducerConfiguration(); - - producerConfig.setRegion(configProps.getProperty(ProducerConfigConstants.AWS_REGION)); + // check and pass the configuration properties + KinesisProducerConfiguration producerConfig = KinesisConfigUtil.validateProducerConfiguration(configProps); producerConfig.setCredentialsProvider(AWSUtil.getCredentialsProvider(configProps)); - if (configProps.containsKey(ProducerConfigConstants.COLLECTION_MAX_COUNT)) { - producerConfig.setCollectionMaxCount(PropertiesUtil.getLong(configProps, - ProducerConfigConstants.COLLECTION_MAX_COUNT, producerConfig.getCollectionMaxCount(), LOG)); - } - if (configProps.containsKey(ProducerConfigConstants.AGGREGATION_MAX_COUNT)) { - producerConfig.setAggregationMaxCount(PropertiesUtil.getLong(configProps, - ProducerConfigConstants.AGGREGATION_MAX_COUNT, producerConfig.getAggregationMaxCount(), LOG)); - } producer = new KinesisProducer(producerConfig); callback = new FutureCallback() { diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java index d131150b1697b..983687ead8ab6 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java @@ -20,14 +20,24 @@ import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; /** + * @deprecated + * * Optional producer specific configuration keys for {@link FlinkKinesisProducer}. */ +@Deprecated public class ProducerConfigConstants extends AWSConfigConstants { - /** Maximum number of items to pack into an PutRecords request. **/ + /** + * @deprecated + * + * Deprecated key. **/ + @Deprecated public static final String COLLECTION_MAX_COUNT = "aws.producer.collectionMaxCount"; - /** Maximum number of items to pack into an aggregated record. **/ + /** + * @deprecated + * + * Deprecated key. **/ + @Deprecated public static final String AGGREGATION_MAX_COUNT = "aws.producer.aggregationMaxCount"; - } diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java index 42f1af055ad82..997191c464f68 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtil.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.producer.KinesisProducerConfiguration; import java.text.ParseException; import java.text.SimpleDateFormat; @@ -38,6 +39,22 @@ * Utilities for Flink Kinesis connector configuration. */ public class KinesisConfigUtil { + + /** Maximum number of items to pack into an PutRecords request. **/ + protected static final String COLLECTION_MAX_COUNT = "CollectionMaxCount"; + + /** Maximum number of items to pack into an aggregated record. **/ + protected static final String AGGREGATION_MAX_COUNT = "AggregationMaxCount"; + + /** Limits the maximum allowed put rate for a shard, as a percentage of the backend limits. + * The default value is set as 100% in Flink. KPL's default value is 150% but it makes KPL throw + * RateLimitExceededException too frequently and breaks Flink sink as a result. + **/ + private static final String RATE_LIMIT = "RateLimit"; + + /** Default values for RateLimit. **/ + private static final String DEFAULT_RATE_LIMIT = "100"; + /** * Validate configuration properties for {@link FlinkKinesisConsumer}. */ @@ -126,19 +143,40 @@ public static void validateConsumerConfiguration(Properties config) { } } + /** + * Replace deprecated configuration properties for {@link FlinkKinesisProducer}. + * This should be remove along with deprecated keys + */ + public static Properties replaceDeprecatedProducerKeys(Properties configProps) { + // Replace deprecated key + if (configProps.containsKey(ProducerConfigConstants.COLLECTION_MAX_COUNT)) { + configProps.setProperty(COLLECTION_MAX_COUNT, + configProps.getProperty(ProducerConfigConstants.COLLECTION_MAX_COUNT)); + configProps.remove(ProducerConfigConstants.COLLECTION_MAX_COUNT); + } + // Replace deprecated key + if (configProps.containsKey(ProducerConfigConstants.AGGREGATION_MAX_COUNT)) { + configProps.setProperty(AGGREGATION_MAX_COUNT, + configProps.getProperty(ProducerConfigConstants.AGGREGATION_MAX_COUNT)); + configProps.remove(ProducerConfigConstants.AGGREGATION_MAX_COUNT); + } + return configProps; + } + /** * Validate configuration properties for {@link FlinkKinesisProducer}. */ - public static void validateProducerConfiguration(Properties config) { + public static KinesisProducerConfiguration validateProducerConfiguration(Properties config) { checkNotNull(config, "config can not be null"); validateAwsConfiguration(config); - validateOptionalPositiveLongProperty(config, ProducerConfigConstants.COLLECTION_MAX_COUNT, - "Invalid value given for maximum number of items to pack into a PutRecords request. Must be a valid non-negative long value."); + // Override KPL default value if it's not specified by user + if (!config.containsKey(RATE_LIMIT)) { + config.setProperty(RATE_LIMIT, DEFAULT_RATE_LIMIT); + } - validateOptionalPositiveLongProperty(config, ProducerConfigConstants.AGGREGATION_MAX_COUNT, - "Invalid value given for maximum number of items to pack into an aggregated record. Must be a valid non-negative long value."); + return KinesisProducerConfiguration.fromProperties(config); } /** diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java index 6af4c622b320a..4a007d5b2c009 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/FlinkKinesisConsumerTest.java @@ -33,7 +33,6 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; -import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; import org.apache.flink.streaming.connectors.kinesis.internals.KinesisDataFetcher; import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShard; import org.apache.flink.streaming.connectors.kinesis.model.KinesisStreamShardState; @@ -46,7 +45,6 @@ import org.apache.flink.streaming.connectors.kinesis.testutils.TestableFlinkKinesisConsumer; import org.apache.flink.streaming.connectors.kinesis.util.KinesisConfigUtil; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; -import org.apache.flink.util.InstantiationUtil; import com.amazonaws.services.kinesis.model.HashKeyRange; import com.amazonaws.services.kinesis.model.SequenceNumberRange; @@ -506,76 +504,6 @@ public void testUnparsableLongForShardDiscoveryIntervalMillisInConfig() { KinesisConfigUtil.validateConsumerConfiguration(testConfig); } - // ---------------------------------------------------------------------- - // FlinkKinesisConsumer.validateProducerConfiguration() tests - // ---------------------------------------------------------------------- - - @Test - public void testUnparsableLongForCollectionMaxCountInConfig() { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Invalid value given for maximum number of items to pack into a PutRecords request"); - - Properties testConfig = new Properties(); - testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1"); - testConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); - testConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); - testConfig.setProperty(ProducerConfigConstants.COLLECTION_MAX_COUNT, "unparsableLong"); - - KinesisConfigUtil.validateProducerConfiguration(testConfig); - } - - @Test - public void testUnparsableLongForAggregationMaxCountInConfig() { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Invalid value given for maximum number of items to pack into an aggregated record"); - - Properties testConfig = new Properties(); - testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1"); - testConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); - testConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); - testConfig.setProperty(ProducerConfigConstants.AGGREGATION_MAX_COUNT, "unparsableLong"); - - KinesisConfigUtil.validateProducerConfiguration(testConfig); - } - - // ---------------------------------------------------------------------- - // Tests to verify serializability - // ---------------------------------------------------------------------- - - @Test - public void testCreateWithNonSerializableDeserializerFails() { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("The provided deserialization schema is not serializable"); - - Properties testConfig = new Properties(); - testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1"); - testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); - testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); - - new FlinkKinesisConsumer<>("test-stream", new NonSerializableDeserializationSchema(), testConfig); - } - - @Test - public void testCreateWithSerializableDeserializer() { - Properties testConfig = new Properties(); - testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1"); - testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); - testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); - - new FlinkKinesisConsumer<>("test-stream", new SerializableDeserializationSchema(), testConfig); - } - - @Test - public void testConsumerIsSerializable() { - Properties testConfig = new Properties(); - testConfig.setProperty(ConsumerConfigConstants.AWS_REGION, "us-east-1"); - testConfig.setProperty(ConsumerConfigConstants.AWS_ACCESS_KEY_ID, "accessKeyId"); - testConfig.setProperty(ConsumerConfigConstants.AWS_SECRET_ACCESS_KEY, "secretKey"); - - FlinkKinesisConsumer consumer = new FlinkKinesisConsumer<>("test-stream", new SimpleStringSchema(), testConfig); - assertTrue(InstantiationUtil.isSerializable(consumer)); - } - // ---------------------------------------------------------------------- // Tests related to state initialization // ---------------------------------------------------------------------- diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java new file mode 100644 index 0000000000000..d14ac04a8f495 --- /dev/null +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.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.streaming.connectors.kinesis.util; + +import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Properties; + +import static org.junit.Assert.assertEquals; + +/** + * Tests for KinesisConfigUtil. + */ +@RunWith(PowerMockRunner.class) +@PrepareForTest({FlinkKinesisConsumer.class, KinesisConfigUtil.class}) +public class KinesisConfigUtilTest { + @Rule + private ExpectedException exception = ExpectedException.none(); + + @Test + public void testUnparsableLongForProducerConfiguration() { + exception.expect(IllegalArgumentException.class); + exception.expectMessage("Error trying to set field RateLimit with the value 'unparsableLong'"); + + Properties testConfig = new Properties(); + testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty("RateLimit", "unparsableLong"); + + KinesisConfigUtil.validateProducerConfiguration(testConfig); + } + + @Test + public void testReplaceDeprecatedKeys() { + Properties testConfig = new Properties(); + testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(ProducerConfigConstants.AGGREGATION_MAX_COUNT, "1"); + testConfig.setProperty(ProducerConfigConstants.COLLECTION_MAX_COUNT, "2"); + Properties replacedConfig = KinesisConfigUtil.replaceDeprecatedProducerKeys(testConfig); + + assertEquals("1", replacedConfig.getProperty(KinesisConfigUtil.AGGREGATION_MAX_COUNT)); + assertEquals("2", replacedConfig.getProperty(KinesisConfigUtil.COLLECTION_MAX_COUNT)); + } +} From 59eab45458b3b1637ccbc5dafd326cc84ffb9655 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 15 Aug 2017 13:46:29 +0800 Subject: [PATCH 109/129] [FLINK-7363] [kinesis] Clean up deprecation of ProducerConfigConstants - Improve deprecation message in Javadocs - Remove usage of ProducerConfigConstants in code wherever possible - Remove usage of ProducerConfigConstants in documentation code snippets This closes #4473. --- docs/dev/connectors/kinesis.md | 36 +++++++++---------- .../config/ProducerConfigConstants.java | 19 ++++++---- .../kinesis/examples/ProduceIntoKinesis.java | 8 ++--- .../ManualConsumerProducerTest.java | 8 ++--- .../manualtests/ManualProducerTest.java | 8 ++--- .../kinesis/util/KinesisConfigUtilTest.java | 6 ++-- 6 files changed, 47 insertions(+), 38 deletions(-) diff --git a/docs/dev/connectors/kinesis.md b/docs/dev/connectors/kinesis.md index 1eea308fd96df..3ffe1c41e7922 100644 --- a/docs/dev/connectors/kinesis.md +++ b/docs/dev/connectors/kinesis.md @@ -271,9 +271,9 @@ For the monitoring to work, the user accessing the stream needs access to the Cl {% highlight java %} Properties producerConfig = new Properties(); // Required configs -producerConfig.put(ProducerConfigConstants.AWS_REGION, "us-east-1"); -producerConfig.put(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); -producerConfig.put(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); +producerConfig.put(AWSConfigConstants.AWS_REGION, "us-east-1"); +producerConfig.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); +producerConfig.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); // Optional configs producerConfig.put("AggregationMaxCount", "4294967295"); producerConfig.put("CollectionMaxCount", "1000"); @@ -293,10 +293,10 @@ simpleStringStream.addSink(kinesis); {% highlight scala %} val producerConfig = new Properties(); // Required configs -producerConfig.put(ProducerConfigConstants.AWS_REGION, "us-east-1"); -producerConfig.put(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); -producerConfig.put(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); -// Optional configs +producerConfig.put(AWSConfigConstants.AWS_REGION, "us-east-1"); +producerConfig.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); +producerConfig.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); +// Optional KPL configs producerConfig.put("AggregationMaxCount", "4294967295"); producerConfig.put("CollectionMaxCount", "1000"); producerConfig.put("RecordTtl", "30000"); @@ -328,29 +328,29 @@ It is sometimes desirable to have Flink operate as a consumer or producer agains [Kinesalite](https://github.com/mhart/kinesalite); this is especially useful when performing functional testing of a Flink application. The AWS endpoint that would normally be inferred by the AWS region set in the Flink configuration must be overridden via a configuration property. -To override the AWS endpoint, taking the producer for example, set the `ProducerConfigConstants.AWS_ENDPOINT` property in the -Flink configuration, in addition to the `ProducerConfigConstants.AWS_REGION` required by Flink. Although the region is +To override the AWS endpoint, taking the producer for example, set the `AWSConfigConstants.AWS_ENDPOINT` property in the +Flink configuration, in addition to the `AWSConfigConstants.AWS_REGION` required by Flink. Although the region is required, it will not be used to determine the AWS endpoint URL. -The following example shows how one might supply the `ProducerConfigConstants.AWS_ENDPOINT` configuration property: +The following example shows how one might supply the `AWSConfigConstants.AWS_ENDPOINT` configuration property:
{% highlight java %} Properties producerConfig = new Properties(); -producerConfig.put(ProducerConfigConstants.AWS_REGION, "us-east-1"); -producerConfig.put(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); -producerConfig.put(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); -producerConfig.put(ProducerConfigConstants.AWS_ENDPOINT, "http://localhost:4567"); +producerConfig.put(AWSConfigConstants.AWS_REGION, "us-east-1"); +producerConfig.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); +producerConfig.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); +producerConfig.put(AWSConfigConstants.AWS_ENDPOINT, "http://localhost:4567"); {% endhighlight %}
{% highlight scala %} val producerConfig = new Properties(); -producerConfig.put(ProducerConfigConstants.AWS_REGION, "us-east-1"); -producerConfig.put(ProducerConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); -producerConfig.put(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); -producerConfig.put(ProducerConfigConstants.AWS_ENDPOINT, "http://localhost:4567"); +producerConfig.put(AWSConfigConstants.AWS_REGION, "us-east-1"); +producerConfig.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); +producerConfig.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); +producerConfig.put(AWSConfigConstants.AWS_ENDPOINT, "http://localhost:4567"); {% endhighlight %}
diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java index 983687ead8ab6..d66bb90f9c8a1 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ProducerConfigConstants.java @@ -20,24 +20,31 @@ import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; /** - * @deprecated - * * Optional producer specific configuration keys for {@link FlinkKinesisProducer}. + * + * @deprecated This class is deprecated in favor of the official AWS Kinesis producer configuration keys. + * See + * here for the full list of available configs. + * For configuring the region and credentials, please use the keys in {@link AWSConfigConstants}. */ @Deprecated public class ProducerConfigConstants extends AWSConfigConstants { /** - * @deprecated + * Deprecated key. * - * Deprecated key. **/ + * @deprecated This is deprecated in favor of the official AWS Kinesis producer configuration keys. + * Please use {@code CollectionMaxCount} instead. + **/ @Deprecated public static final String COLLECTION_MAX_COUNT = "aws.producer.collectionMaxCount"; /** - * @deprecated + * Deprecated key. * - * Deprecated key. **/ + * @deprecated This is deprecated in favor of the official AWS Kinesis producer configuration keys. + * Please use {@code AggregationMaxCount} instead. + **/ @Deprecated public static final String AGGREGATION_MAX_COUNT = "aws.producer.aggregationMaxCount"; } diff --git a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java index ee031eb80b1b9..8d21c2caa1f91 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java +++ b/flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/examples/ProduceIntoKinesis.java @@ -22,7 +22,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; -import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.apache.commons.lang3.RandomStringUtils; @@ -43,9 +43,9 @@ public static void main(String[] args) throws Exception { DataStream simpleStringStream = see.addSource(new EventsGenerator()); Properties kinesisProducerConfig = new Properties(); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region")); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey")); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_REGION, pt.getRequired("region")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey")); FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>( new SimpleStringSchema(), kinesisProducerConfig); diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java index 2915e2f6da1b3..a7470dc166e2f 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualConsumerProducerTest.java @@ -25,8 +25,8 @@ import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; -import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; import org.apache.flink.streaming.connectors.kinesis.examples.ProduceIntoKinesis; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; @@ -56,9 +56,9 @@ public static void main(String[] args) throws Exception { DataStream simpleStringStream = see.addSource(new ProduceIntoKinesis.EventsGenerator()); Properties kinesisProducerConfig = new Properties(); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region")); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey")); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_REGION, pt.getRequired("region")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey")); FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>( new KinesisSerializationSchema() { diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java index 8abf4bb2ef655..fb49169bc8110 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualProducerTest.java @@ -23,7 +23,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisProducer; import org.apache.flink.streaming.connectors.kinesis.KinesisPartitioner; -import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; import org.apache.flink.streaming.connectors.kinesis.examples.ProduceIntoKinesis; import org.apache.flink.streaming.connectors.kinesis.serialization.KinesisSerializationSchema; @@ -53,9 +53,9 @@ public static void main(String[] args) throws Exception { DataStream simpleStringStream = see.addSource(new ProduceIntoKinesis.EventsGenerator()); Properties kinesisProducerConfig = new Properties(); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_REGION, pt.getRequired("region")); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey")); - kinesisProducerConfig.setProperty(ProducerConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_REGION, pt.getRequired("region")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_ACCESS_KEY_ID, pt.getRequired("accessKey")); + kinesisProducerConfig.setProperty(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, pt.getRequired("secretKey")); FlinkKinesisProducer kinesis = new FlinkKinesisProducer<>( new KinesisSerializationSchema() { diff --git a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java index d14ac04a8f495..3b000588f4b91 100644 --- a/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java +++ b/flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigUtilTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.kinesis.util; import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; import org.junit.Rule; @@ -46,7 +47,7 @@ public void testUnparsableLongForProducerConfiguration() { exception.expectMessage("Error trying to set field RateLimit with the value 'unparsableLong'"); Properties testConfig = new Properties(); - testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1"); testConfig.setProperty("RateLimit", "unparsableLong"); KinesisConfigUtil.validateProducerConfiguration(testConfig); @@ -55,7 +56,8 @@ public void testUnparsableLongForProducerConfiguration() { @Test public void testReplaceDeprecatedKeys() { Properties testConfig = new Properties(); - testConfig.setProperty(ProducerConfigConstants.AWS_REGION, "us-east-1"); + testConfig.setProperty(AWSConfigConstants.AWS_REGION, "us-east-1"); + // these deprecated keys should be replaced testConfig.setProperty(ProducerConfigConstants.AGGREGATION_MAX_COUNT, "1"); testConfig.setProperty(ProducerConfigConstants.COLLECTION_MAX_COUNT, "2"); Properties replacedConfig = KinesisConfigUtil.replaceDeprecatedProducerKeys(testConfig); From fb062773d5f0a65b364ed57936c6865801ed33be Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Tue, 12 Sep 2017 16:59:37 +0800 Subject: [PATCH 110/129] Fix merge error --- pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/pom.xml b/pom.xml index b29e207d31d1e..2d2750f96fbe6 100644 --- a/pom.xml +++ b/pom.xml @@ -84,13 +84,8 @@ under the License. UTF-8 UTF-8 -<<<<<<< HEAD - 2.4.1 -======= - never-match-me 2.8.0 ->>>>>>> 76d80f86be9162b32581306d489fd58fcc86f6ca From 530368cb1042b6e77142ce7b1189bbda1a538150 Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Thu, 14 Sep 2017 11:52:35 +0800 Subject: [PATCH 111/129] fix merge error in pom.xml --- pom.xml | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index b29e207d31d1e..561ec791868dc 100644 --- a/pom.xml +++ b/pom.xml @@ -67,7 +67,7 @@ under the License. flink-connectors flink-examples flink-clients - + flink-tests flink-test-utils-parent flink-libraries flink-scala-shell @@ -84,13 +84,7 @@ under the License. UTF-8 UTF-8 -<<<<<<< HEAD 2.4.1 -======= - - never-match-me - 2.8.0 ->>>>>>> 76d80f86be9162b32581306d489fd58fcc86f6ca From 7219503d76a6f85798e3dc3602f8318f8f37c0db Mon Sep 17 00:00:00 2001 From: Joe Zhu Date: Thu, 15 Jun 2017 16:27:52 +0800 Subject: [PATCH 112/129] [FLINK-7474] [ Streaming Connectors] AzureEventhubs-connector, support read from and write to Azure eventhubs --- .../flink-connector-eventhubs/pom.xml | 141 +++++++ .../eventhubs/FlinkEventHubConsumer.java | 309 +++++++++++++++ .../eventhubs/FlinkEventHubProducer.java | 110 ++++++ .../eventhubs/internals/EventFetcher.java | 370 ++++++++++++++++++ .../internals/EventhubClientWrapper.java | 200 ++++++++++ .../internals/EventhubConsumerThread.java | 128 ++++++ .../internals/EventhubOffsetType.java | 12 + .../internals/EventhubPartition.java | 70 ++++ .../internals/EventhubPartitionState.java | 29 ++ ...bPartitionStateWithPeriodicWatermarks.java | 38 ++ ...artitionStateWithPunctuatedWatermarks.java | 48 +++ .../internals/EventhubProducerThread.java | 95 +++++ .../eventhubs/internals/Handover.java | 108 +++++ .../eventhubs/internals/ProducerCache.java | 124 ++++++ flink-connectors/pom.xml | 1 + .../flink-examples-streaming/pom.xml | 5 + .../examples/eventhub/ReadFromEventhub.java | 40 ++ .../examples/eventhub/WriteToEventhub.java | 53 +++ 18 files changed, 1881 insertions(+) create mode 100644 flink-connectors/flink-connector-eventhubs/pom.xml create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java create mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/ReadFromEventhub.java create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/WriteToEventhub.java diff --git a/flink-connectors/flink-connector-eventhubs/pom.xml b/flink-connectors/flink-connector-eventhubs/pom.xml new file mode 100644 index 0000000000000..70e3c075a7ad8 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/pom.xml @@ -0,0 +1,141 @@ + + + + flink-connectors + org.apache.flink + 1.4-SNAPSHOT + + 4.0.0 + + flink-connector-azureeventhubs_${scala.binary.version} + flink-connector-azureeventhubs + + jar + + + + + + com.microsoft.azure + azure-eventhubs + 0.14.0 + provided + + + + org.apache.flink + flink-connector-kafka-base_${scala.binary.version} + ${project.version} + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + provided + + true + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + test + test-jar + + + + org.apache.flink + flink-tests_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${project.version} + test-jar + test + + + + org.apache.flink + flink-metrics-jmx + ${project.version} + test + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-test-sources + + test-jar-no-fork + + + + **/KafkaTestEnvironmentImpl* + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + -Xms256m -Xmx1000m -Dlog4j.configuration=${log4j.configuration} -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit + + + + + + + diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java new file mode 100644 index 0000000000000..ba4abd7bfdd5d --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -0,0 +1,309 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.ClosureCleaner; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.DefaultOperatorStateBackend; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.connectors.eventhubs.internals.EventFetcher; +import org.apache.flink.streaming.connectors.eventhubs.internals.EventhubPartition; +import org.apache.flink.streaming.util.serialization.DeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchemaWrapper; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedValue; + +import com.microsoft.azure.eventhubs.PartitionReceiver; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +/** + * Created by jozh on 5/22/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + * This class is used to create datastream from event hub + */ + +public class FlinkEventHubConsumer extends RichParallelSourceFunction implements + CheckpointedFunction, + ResultTypeQueryable, + CheckpointedRestoring> { + private static final long serialVersionUID = -3247976470793561346L; + protected static final Logger LOGGER = LoggerFactory.getLogger(FlinkEventHubConsumer.class); + protected static final String DEFAULTOFFSETSTATENAME = "flink.eventhub.offset"; + + protected final KeyedDeserializationSchema deserializer; + protected final Properties eventhubsProps; + protected final int partitionCount; + protected List> subscribedPartitions; + protected final String defaultEventhubInitOffset; + + private Map subscribedPartitionsToStartOffsets; + private SerializedValue> periodicWatermarkAssigner; + private SerializedValue> punctuatedWatermarkAssigner; + private transient ListState> offsetsStateForCheckpoint; + private transient volatile EventFetcher eventhubFetcher; + private transient volatile HashMap restoreToOffset; + private volatile boolean running = true; + + private Counter receivedCount; + + public FlinkEventHubConsumer(Properties eventhubsProps, DeserializationSchema deserializer){ + this(eventhubsProps, new KeyedDeserializationSchemaWrapper(deserializer)); + } + + public FlinkEventHubConsumer(Properties eventhubsProps, KeyedDeserializationSchema deserializer){ + Preconditions.checkNotNull(eventhubsProps); + Preconditions.checkNotNull(deserializer); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policyname")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policykey")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.namespace")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.name")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.partition.count")); + + this.eventhubsProps = eventhubsProps; + this.partitionCount = Integer.parseInt(eventhubsProps.getProperty("eventhubs.partition.count")); + this.deserializer = deserializer; + + String userDefinedOffset = eventhubsProps.getProperty("eventhubs.auto.offset"); + if (userDefinedOffset != null && userDefinedOffset.toLowerCase().compareTo("lastest") == 0){ + this.defaultEventhubInitOffset = PartitionReceiver.END_OF_STREAM; + } + else { + this.defaultEventhubInitOffset = PartitionReceiver.START_OF_STREAM; + } + + if (this.partitionCount <= 0){ + throw new IllegalArgumentException("eventhubs.partition.count must greater than 0"); + } + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + receivedCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("received_event_count"); + + List eventhubPartitions = this.getAllEventhubPartitions(); + this.subscribedPartitionsToStartOffsets = new HashMap<>(eventhubPartitions.size()); + + if (this.restoreToOffset != null){ + for (EventhubPartition partition : eventhubPartitions){ + if (this.restoreToOffset.containsKey(partition)){ + this.subscribedPartitionsToStartOffsets.put(partition, restoreToOffset.get(partition)); + } + } + + LOGGER.info("Consumer subtask {} will start reading {} partitions with offsets in restored state: {}", + getRuntimeContext().getIndexOfThisSubtask(), + this.subscribedPartitionsToStartOffsets.size(), + this.subscribedPartitionsToStartOffsets); + } + else { + //If there is no restored state. Then all partitions to read from start, the offset is "-1". In the + //future eventhub supports specify offset, we modify here + //We assign partition to each subTask in round robin mode + int numParallelSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); + int indexofThisSubtask = getRuntimeContext().getIndexOfThisSubtask(); + for (int i = 0; i < eventhubPartitions.size(); i++) { + if (i % numParallelSubtasks == indexofThisSubtask) { + this.subscribedPartitionsToStartOffsets.put(eventhubPartitions.get(i), defaultEventhubInitOffset); + } + } + + LOGGER.info("Consumer subtask {} will start reading {} partitions with offsets: {}", + getRuntimeContext().getIndexOfThisSubtask(), + this.subscribedPartitionsToStartOffsets.size(), + this.subscribedPartitionsToStartOffsets); + } + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + if (this.subscribedPartitionsToStartOffsets == null || this.subscribedPartitionsToStartOffsets.size() == 0){ + throw new Exception("The partitions were not set for the consumer"); + } + + StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext(); + + if (!this.subscribedPartitionsToStartOffsets.isEmpty()){ + final EventFetcher fetcher = new EventFetcher(sourceContext, + subscribedPartitionsToStartOffsets, + deserializer, + periodicWatermarkAssigner, + punctuatedWatermarkAssigner, + runtimeContext.getProcessingTimeService(), + runtimeContext.getExecutionConfig().getAutoWatermarkInterval(), + runtimeContext.getUserCodeClassLoader(), + runtimeContext.getTaskNameWithSubtasks(), + eventhubsProps, + false, + receivedCount); + + this.eventhubFetcher = fetcher; + if (!this.running){ + return; + } + + this.eventhubFetcher.runFetchLoop(); + } + else { + sourceContext.emitWatermark(new Watermark(Long.MAX_VALUE)); + + final Object waitObj = new Object(); + while (this.running){ + try { + synchronized (waitObj){ + waitObj.wait(); + } + } + catch (InterruptedException ex){ + if (this.running){ + Thread.currentThread().interrupt(); + } + } + } + } + } + + @Override + public void close() throws Exception { + try { + this.cancel(); + } + finally { + super.close(); + } + } + + @Override + public void cancel() { + this.running = false; + + if (this.eventhubFetcher != null){ + this.eventhubFetcher.cancel(); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (!this.running){ + LOGGER.info("Consumer subtask {}: snapshotState() is called on the closed source", getRuntimeContext().getIndexOfThisSubtask()); + return; + } + + this.offsetsStateForCheckpoint.clear(); + final EventFetcher fetcher = this.eventhubFetcher; + if (fetcher == null){ + for (Map.Entry subscribedPartition : this.subscribedPartitionsToStartOffsets.entrySet()){ + this.offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue())); + } + } + else { + HashMap currentOffsets = fetcher.snapshotCurrentState(); + for (Map.Entry subscribedPartition : currentOffsets.entrySet()){ + this.offsetsStateForCheckpoint.add(Tuple2.of(subscribedPartition.getKey(), subscribedPartition.getValue())); + } + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOGGER.info("Consumer subtask {}:Start init eventhub offset state", getRuntimeContext().getIndexOfThisSubtask()); + OperatorStateStore stateStore = context.getOperatorStateStore(); + /* this.offsetsStateForCheckpoint = stateStore + .getListState(new ListStateDescriptor>(DEFAULT_OFFSET_STATE_NAME, TypeInformation.of(new TypeHint>(){}))); +*/ + this.offsetsStateForCheckpoint = stateStore.getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME); + if (context.isRestored()){ + if (this.restoreToOffset == null){ + this.restoreToOffset = new HashMap<>(); + for (Tuple2 offsetState : this.offsetsStateForCheckpoint.get()){ + this.restoreToOffset.put(offsetState.f0, offsetState.f1); + } + + LOGGER.info("Consumer subtask {}:Eventhub offset state is restored from checkpoint", getRuntimeContext().getIndexOfThisSubtask()); + } + else if (this.restoreToOffset.isEmpty()){ + this.restoreToOffset = null; + } + } + else { + LOGGER.info("Consumer subtask {}:No restore state for flink-eventhub-consumer", getRuntimeContext().getIndexOfThisSubtask()); + } + } + + @Override + public void restoreState(HashMap eventhubPartitionOffsets) throws Exception { + LOGGER.info("{} (taskIdx={}) restoring offsets from an older version.", + getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask()); + + this.restoreToOffset = eventhubPartitionOffsets; + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("{} (taskIdx={}) restored offsets from an older Flink version: {}", + getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask(), eventhubPartitionOffsets); + } + } + + @Override + public TypeInformation getProducedType() { + return this.deserializer.getProducedType(); + } + + public FlinkEventHubConsumer assignTimestampsAndWatermarks(AssignerWithPunctuatedWatermarks assigner) { + Preconditions.checkNotNull(assigner); + + if (this.periodicWatermarkAssigner != null) { + throw new IllegalStateException("A periodic watermark emitter has already been set."); + } + try { + ClosureCleaner.clean(assigner, true); + this.punctuatedWatermarkAssigner = new SerializedValue<>(assigner); + return this; + } catch (Exception e) { + throw new IllegalArgumentException("The given assigner is not serializable", e); + } + } + + public FlinkEventHubConsumer assignTimestampsAndWatermarks(AssignerWithPeriodicWatermarks assigner) { + Preconditions.checkNotNull(assigner); + + if (this.punctuatedWatermarkAssigner != null) { + throw new IllegalStateException("A punctuated watermark emitter has already been set."); + } + try { + ClosureCleaner.clean(assigner, true); + this.periodicWatermarkAssigner = new SerializedValue<>(assigner); + return this; + } catch (Exception e) { + throw new IllegalArgumentException("The given assigner is not serializable", e); + } + } + + private List getAllEventhubPartitions() { + List partitions = new ArrayList<>(); + for (int i = 0; i < this.partitionCount; i++){ + partitions.add(new EventhubPartition(this.eventhubsProps, i)); + } + + LOGGER.info("Consumer subtask {}:Create {} eventhub partitions info", getRuntimeContext().getIndexOfThisSubtask(), this.partitionCount); + return partitions; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java new file mode 100644 index 0000000000000..61db348869c5f --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java @@ -0,0 +1,110 @@ +package org.apache.flink.streaming.connectors.eventhubs; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.streaming.connectors.eventhubs.internals.EventhubProducerThread; +import org.apache.flink.streaming.connectors.eventhubs.internals.ProducerCache; +import org.apache.flink.streaming.util.serialization.SerializationSchema; +import org.apache.flink.util.Preconditions; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +/** + * Created by jozh on 6/20/2017. + * Will support customize parttion in next version + */ +public class FlinkEventHubProducer extends RichSinkFunction implements CheckpointedFunction { + + private static final Logger logger = LoggerFactory.getLogger(FlinkEventHubProducer.class); + private static final long serialVersionUID = -7486455932880508035L; + private final SerializationSchema schema; + private final ProducerCache cache; + private final Properties eventhubsProps; + private EventhubProducerThread producerThread; + private Counter prepareSendCount; + private Counter commitSendCount; + + public FlinkEventHubProducer(SerializationSchema serializationSchema, Properties eventhubsProps){ + Preconditions.checkNotNull(serializationSchema); + Preconditions.checkNotNull(eventhubsProps); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policyname")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.policykey")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.namespace")); + Preconditions.checkNotNull(eventhubsProps.getProperty("eventhubs.name")); + + this.schema = serializationSchema; + this.eventhubsProps = eventhubsProps; + + int capacity = eventhubsProps.getProperty("eventhubs.cache.capacity") == null + ? ProducerCache.DEFAULTCAPACITY : Integer.parseInt(eventhubsProps.getProperty("eventhubs.cache.capacity")); + + long timeout = eventhubsProps.getProperty("eventhubs.cache.timeout") == null + ? ProducerCache.DEFAULTTIMEOUTMILLISECOND : Long.parseLong(eventhubsProps.getProperty("eventhubs.cache.timeout")); + + this.cache = new ProducerCache(capacity, timeout); + + logger.info("Created eventhub producer for namespace: {}, name: {}", + eventhubsProps.getProperty("eventhubs.namespace"), + eventhubsProps.getProperty("eventhubs.name")); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + return; + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + return; + } + + @Override + public void invoke(OUT value) throws Exception { + cache.checkErr(); + EventData event = new EventData(this.schema.serialize(value)); + cache.put(event); + prepareSendCount.inc(); + logger.debug("Insert a event input output cache"); + cache.checkErr(); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + prepareSendCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("prepare_send_event_count"); + commitSendCount = getRuntimeContext().getMetricGroup().addGroup(this.getClass().getName()).counter("commit_send_event_count"); + String threadName = getEventhubProducerName(); + + logger.info("Eventhub producer thread {} starting", threadName); + producerThread = new EventhubProducerThread( + logger, + threadName, + cache, + eventhubsProps, + commitSendCount); + producerThread.start(); + logger.info("Eventhub producer thread {} started", threadName); + cache.checkErr(); + } + + @Override + public void close() throws Exception { + super.close(); + logger.info("Eventhub producer thread close on demand"); + producerThread.shutdown(); + cache.close(); + cache.checkErr(); + } + + protected String getEventhubProducerName(){ + return "Eventhub producer " + getRuntimeContext().getTaskNameWithSubtasks(); + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java new file mode 100644 index 0000000000000..cae6327052930 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java @@ -0,0 +1,370 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.metrics.Counter; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.util.SerializedValue; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Created by jozh on 6/14/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector. + * A fetcher that fetches data from Eventhub via the EventhubUtil. + * Eventhub offset is stored at flink checkpoint backend + * @param The type of elements produced by the fetcher. + */ +public class EventFetcher { + protected static final int NO_TIMESTAMPS_WATERMARKS = 0; + protected static final int PERIODIC_WATERMARKS = 1; + protected static final int PUNCTUATED_WATERMARKS = 2; + private static final Logger logger = LoggerFactory.getLogger(EventFetcher.class); + private volatile boolean running = true; + + private final KeyedDeserializationSchema deserializer; + private final Handover handover; + private final Properties eventhubProps; + private final EventhubConsumerThread consumerThread; + private final String taskNameWithSubtasks; + + + protected final SourceFunction.SourceContext sourceContext; + protected final Object checkpointLock; + private final Map subscribedPartitionStates; + protected final int timestampWatermarkMode; + protected final boolean useMetrics; + private volatile long maxWatermarkSoFar = Long.MIN_VALUE; + private Counter receivedCount; + + public EventFetcher( + SourceFunction.SourceContext sourceContext, + Map assignedPartitionsWithInitialOffsets, + KeyedDeserializationSchema deserializer, + SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + ProcessingTimeService processTimerProvider, + long autoWatermarkInterval, + ClassLoader userCodeClassLoader, + String taskNameWithSubtasks, + Properties eventhubProps, + boolean useMetrics, + Counter receivedCount) throws Exception { + + this.sourceContext = checkNotNull(sourceContext); + this.deserializer = checkNotNull(deserializer); + this.eventhubProps = eventhubProps; + this.checkpointLock = sourceContext.getCheckpointLock(); + this.useMetrics = useMetrics; + this.receivedCount = receivedCount; + this.taskNameWithSubtasks = taskNameWithSubtasks; + this.timestampWatermarkMode = getTimestampWatermarkMode(watermarksPeriodic, watermarksPunctuated); + + this.subscribedPartitionStates = initializeSubscribedPartitionStates( + assignedPartitionsWithInitialOffsets, + timestampWatermarkMode, + watermarksPeriodic, watermarksPunctuated, + userCodeClassLoader); + + this.handover = new Handover(); + this.consumerThread = new EventhubConsumerThread(logger, + handover, + this.eventhubProps, + getFetcherName() + " for " + taskNameWithSubtasks, + this.subscribedPartitionStates.values().toArray(new EventhubPartitionState[this.subscribedPartitionStates.size()])); + + if (this.timestampWatermarkMode == PERIODIC_WATERMARKS) { + PeriodicWatermarkEmitter periodicEmitter = + new PeriodicWatermarkEmitter(this.subscribedPartitionStates, sourceContext, processTimerProvider, autoWatermarkInterval); + periodicEmitter.start(); + } + + } + + public HashMap snapshotCurrentState() { + // this method assumes that the checkpoint lock is held + logger.debug("snapshot current offset state for subtask {}", taskNameWithSubtasks); + assert Thread.holdsLock(checkpointLock); + + HashMap state = new HashMap<>(subscribedPartitionStates.size()); + for (Map.Entry partition : subscribedPartitionStates.entrySet()){ + state.put(partition.getKey(), partition.getValue().getOffset()); + } + + return state; + } + + public void runFetchLoop() throws Exception{ + try { + final Handover handover = this.handover; + consumerThread.start(); + logger.info("Eventhub consumer thread started for substask {}", taskNameWithSubtasks); + + logger.info("Start fetcher loop to get data from eventhub and emit to flink for subtask {}", taskNameWithSubtasks); + while (running){ + final Tuple2> eventsTuple = handover.pollNext(); + for (EventData event : eventsTuple.f1){ + final T value = deserializer.deserialize(null, + event.getBytes(), + event.getSystemProperties().getPartitionKey(), + eventsTuple.f0.getParitionId(), + event.getSystemProperties().getSequenceNumber()); + + if (deserializer.isEndOfStream(value)){ + running = false; + break; + } + emitRecord(value, subscribedPartitionStates.get(eventsTuple.f0), event.getSystemProperties().getOffset()); + receivedCount.inc(); + } + } + } + finally { + logger.warn("Stopping eventhub consumer thread of subtask {}, because something wrong when deserializing received event " + , taskNameWithSubtasks); + consumerThread.shutdown(); + } + + try { + consumerThread.join(); + logger.warn("Waiting eventhub consumer thread of subtask {} stopped", taskNameWithSubtasks); + } + catch (InterruptedException ex){ + Thread.currentThread().interrupt(); + } + + logger.info("EventFetcher of subtask {} stopped", taskNameWithSubtasks); + } + + public void cancel(){ + logger.info("EventFetcher of subtask {} canceled on demand", taskNameWithSubtasks); + running = false; + handover.close(); + consumerThread.shutdown(); + } + + protected void emitRecord(T record, EventhubPartitionState partitionState, String offset) throws Exception{ + if (record == null){ + synchronized (this.checkpointLock){ + partitionState.setOffset(offset); + } + return; + } + + if (timestampWatermarkMode == NO_TIMESTAMPS_WATERMARKS){ + synchronized (this.checkpointLock){ + sourceContext.collect(record); + partitionState.setOffset(offset); + } + } + else if (timestampWatermarkMode == PERIODIC_WATERMARKS){ + emitRecordWithTimestampAndPeriodicWatermark(record, partitionState, offset, Long.MIN_VALUE); + } + else { + emitRecordWithTimestampAndPunctuatedWatermark(record, partitionState, offset, Long.MIN_VALUE); + } + } + + protected void emitRecordWithTimestampAndPunctuatedWatermark( + T record, + EventhubPartitionState partitionState, + String offset, + long eventTimestamp) { + + final EventhubPartitionStateWithPeriodicWatermarks withWatermarksState = + (EventhubPartitionStateWithPeriodicWatermarks) partitionState; + + final long timestamp; + synchronized (withWatermarksState) { + timestamp = withWatermarksState.getTimestampForRecord(record, eventTimestamp); + } + + synchronized (checkpointLock) { + sourceContext.collectWithTimestamp(record, timestamp); + partitionState.setOffset(offset); + } + } + + protected void emitRecordWithTimestampAndPeriodicWatermark( + T record, + EventhubPartitionState partitionState, + String offset, + long eventTimestamp) { + + final EventhubPartitionStateWithPunctuatedWatermarks withWatermarksState = + (EventhubPartitionStateWithPunctuatedWatermarks) partitionState; + + final long timestamp = withWatermarksState.getTimestampForRecord(record, eventTimestamp); + final Watermark newWatermark = withWatermarksState.checkAndGetNewWatermark(record, timestamp); + + synchronized (checkpointLock) { + sourceContext.collectWithTimestamp(record, timestamp); + partitionState.setOffset(offset); + } + + if (newWatermark != null) { + updateMinPunctuatedWatermark(newWatermark); + } + } + + protected String getFetcherName() { + return "Eventhubs Fetcher"; + } + + private int getTimestampWatermarkMode(SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated) + throws IllegalArgumentException { + if (watermarksPeriodic == null){ + if (watermarksPunctuated == null){ + return NO_TIMESTAMPS_WATERMARKS; + } + else { + return PUNCTUATED_WATERMARKS; + } + } + else { + if (watermarksPunctuated == null){ + return PERIODIC_WATERMARKS; + } + else { + throw new IllegalArgumentException("Cannot have both periodic and punctuated watermarks"); + } + } + } + + private Map initializeSubscribedPartitionStates( + Map assignedPartitionsWithInitialOffsets, + int timestampWatermarkMode, + SerializedValue> watermarksPeriodic, + SerializedValue> watermarksPunctuated, + ClassLoader userCodeClassLoader) throws IOException, ClassNotFoundException { + + if (timestampWatermarkMode != NO_TIMESTAMPS_WATERMARKS + && timestampWatermarkMode != PERIODIC_WATERMARKS + && timestampWatermarkMode != PUNCTUATED_WATERMARKS) { + throw new RuntimeException(); + } + + Map partitionsState = new HashMap<>(assignedPartitionsWithInitialOffsets.size()); + for (Map.Entry partition : assignedPartitionsWithInitialOffsets.entrySet()){ + switch (timestampWatermarkMode){ + case NO_TIMESTAMPS_WATERMARKS:{ + partitionsState.put(partition.getKey(), new EventhubPartitionState(partition.getKey(), partition.getValue())); + logger.info("NO_TIMESTAMPS_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue()); + break; + } + + case PERIODIC_WATERMARKS:{ + AssignerWithPeriodicWatermarks assignerInstance = + watermarksPeriodic.deserializeValue(userCodeClassLoader); + partitionsState.put(partition.getKey(), + new EventhubPartitionStateWithPeriodicWatermarks(partition.getKey(), partition.getValue(), assignerInstance)); + logger.info("PERIODIC_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue()); + break; + } + + case PUNCTUATED_WATERMARKS: { + AssignerWithPunctuatedWatermarks assignerInstance = + watermarksPunctuated.deserializeValue(userCodeClassLoader); + partitionsState.put(partition.getKey(), + new EventhubPartitionStateWithPunctuatedWatermarks(partition.getKey(), partition.getValue(), assignerInstance)); + logger.info("PUNCTUATED_WATERMARKS: Assigned partition {}, offset is {}", partition.getKey(), partition.getValue()); + break; + } + } + } + return partitionsState; + } + + private void updateMinPunctuatedWatermark(Watermark nextWatermark) { + if (nextWatermark.getTimestamp() > maxWatermarkSoFar) { + long newMin = Long.MAX_VALUE; + + for (Map.Entry partition : subscribedPartitionStates.entrySet()){ + final EventhubPartitionStateWithPunctuatedWatermarks withWatermarksState = + (EventhubPartitionStateWithPunctuatedWatermarks) partition.getValue(); + + newMin = Math.min(newMin, withWatermarksState.getCurrentPartitionWatermark()); + } + + // double-check locking pattern + if (newMin > maxWatermarkSoFar) { + synchronized (checkpointLock) { + if (newMin > maxWatermarkSoFar) { + maxWatermarkSoFar = newMin; + sourceContext.emitWatermark(new Watermark(newMin)); + } + } + } + } + } + + private static class PeriodicWatermarkEmitter implements ProcessingTimeCallback { + + private final Map allPartitions; + + private final SourceFunction.SourceContext emitter; + + private final ProcessingTimeService timerService; + + private final long interval; + + private long lastWatermarkTimestamp; + + //------------------------------------------------- + + PeriodicWatermarkEmitter( + Map allPartitions, + SourceFunction.SourceContext emitter, + ProcessingTimeService timerService, + long autoWatermarkInterval) { + this.allPartitions = checkNotNull(allPartitions); + this.emitter = checkNotNull(emitter); + this.timerService = checkNotNull(timerService); + this.interval = autoWatermarkInterval; + this.lastWatermarkTimestamp = Long.MIN_VALUE; + } + + public void start() { + timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this); + } + + @Override + public void onProcessingTime(long timestamp) throws Exception { + + long minAcrossAll = Long.MAX_VALUE; + for (Map.Entry partition : allPartitions.entrySet()){ + final long curr; + EventhubPartitionStateWithPeriodicWatermarks state = + (EventhubPartitionStateWithPeriodicWatermarks) partition.getValue(); + + synchronized (state) { + curr = state.getCurrentWatermarkTimestamp(); + } + + minAcrossAll = Math.min(minAcrossAll, curr); + } + + if (minAcrossAll > lastWatermarkTimestamp) { + lastWatermarkTimestamp = minAcrossAll; + emitter.emitWatermark(new Watermark(minAcrossAll)); + } + + timerService.registerTimer(timerService.getCurrentProcessingTime() + interval, this); + } + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java new file mode 100644 index 0000000000000..9470482ccf5d0 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java @@ -0,0 +1,200 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.util.Preconditions; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.eventhubs.PartitionReceiver; +import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.ServiceBusException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.time.Duration; +import java.time.Instant; +import java.util.Properties; +import java.util.concurrent.ExecutionException; + +/** + * Created by jozh on 6/14/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ +public class EventhubClientWrapper implements Serializable { + private static final long serialVersionUID = -5319150387753930840L; + private static final Logger logger = LoggerFactory.getLogger(EventhubClientWrapper.class); + private EventHubClient eventHubClient; + private PartitionReceiver eventhubReceiver; + private ConnectionStringBuilder connectionString; + private String consumerGroup; + private Long receiverEpoch; + + private Duration receiverTimeout; + private EventhubOffsetType offsetType; + private String currentOffset; + private String partitionId; + + private final int minPrefetchCount = 10; + private int maxPrefetchCount = 999; + private int maxEventRate = 0; + private final Long defaultReceiverEpoch = -1L; + private final String defaultReceiverTimeout = "60000"; + + public void createReveiver(Properties eventhubParams, String partitionId) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10")); + this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM); + } + + public void createReveiver(Properties eventhubParams, String partitionId, String offset) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10")); + this.createReveiver(eventhubParams, partitionId, maxEventRate, offset); + } + + public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM); + } + + /*Will not implement a standalone offset store here, will leverage flink state to save the offset of eventhub*/ + public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate, String offset) + throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ + if (eventhubParams.containsKey("eventhubs.uri") && eventhubParams.containsKey("eventhubs.namespace")) { + throw new IllegalArgumentException("Eventhubs URI and namespace cannot both be specified at the same time."); + } + + if (eventhubParams.containsKey("eventhubs.namespace")){ + this.connectionString = new ConnectionStringBuilder( + eventhubParams.getProperty("eventhubs.namespace"), + eventhubParams.getProperty("eventhubs.name"), + eventhubParams.getProperty("eventhubs.policyname"), + eventhubParams.getProperty("eventhubs.policykey")); + } + else if (eventhubParams.containsKey("eventhubs.uri")){ + this.connectionString = new ConnectionStringBuilder(new URI( + eventhubParams.getProperty("eventhubs.uri")), + eventhubParams.getProperty("eventhubs.name"), + eventhubParams.getProperty("eventhubs.policyname"), + eventhubParams.getProperty("eventhubs.policykey")); + } + else { + throw new IllegalArgumentException("Either Eventhubs URI or namespace nust be specified."); + } + + this.partitionId = Preconditions.checkNotNull(partitionId, "partitionId is no valid, cannot be null or empty"); + this.consumerGroup = eventhubParams.getProperty("eventhubs.consumergroup", EventHubClient.DEFAULT_CONSUMER_GROUP_NAME); + this.receiverEpoch = Long.parseLong(eventhubParams.getProperty("eventhubs.epoch", defaultReceiverEpoch.toString())); + this.receiverTimeout = Duration.ofMillis(Long.parseLong(eventhubParams.getProperty("eventhubs.receiver.timeout", defaultReceiverTimeout))); + this.offsetType = EventhubOffsetType.None; + this.currentOffset = PartitionReceiver.START_OF_STREAM; + + String previousOffset = offset; + + if (previousOffset.compareTo(PartitionReceiver.START_OF_STREAM) != 0 && previousOffset != null) { + + offsetType = EventhubOffsetType.PreviousCheckpoint; + currentOffset = previousOffset; + + } else if (eventhubParams.containsKey("eventhubs.filter.offset")) { + + offsetType = EventhubOffsetType.InputByteOffset; + currentOffset = eventhubParams.getProperty("eventhubs.filter.offset"); + + } else if (eventhubParams.containsKey("eventhubs.filter.enqueuetime")) { + + offsetType = EventhubOffsetType.InputTimeOffset; + currentOffset = eventhubParams.getProperty("eventhubs.filter.enqueuetime"); + } + + this.maxEventRate = maxEventRate; + + if (maxEventRate > 0 && maxEventRate < minPrefetchCount) { + maxPrefetchCount = minPrefetchCount; + } + else if (maxEventRate >= minPrefetchCount && maxEventRate < maxPrefetchCount) { + maxPrefetchCount = maxEventRate + 1; + } + else { + maxEventRate = maxPrefetchCount - 1; + } + + this.createReceiverInternal(); + } + + public Iterable receive () throws ExecutionException, InterruptedException { + return this.eventhubReceiver.receive(maxEventRate).get(); + } + + public void close(){ + logger.info("Close eventhub client on demand of partition {}", this.partitionId); + if (this.eventhubReceiver != null){ + try { + this.eventhubReceiver.closeSync(); + } + catch (ServiceBusException ex){ + logger.error("Close eventhub client of partition {} failed, reason: {}", this.partitionId, ex.getMessage()); + } + } + } + + private void createReceiverInternal() throws IOException, ServiceBusException{ + this.eventHubClient = EventHubClient.createFromConnectionStringSync(this.connectionString.toString()); + + switch (this.offsetType){ + case None: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case PreviousCheckpoint: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case InputByteOffset: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); + } + break; + } + case InputTimeOffset: { + if (this.receiverEpoch > defaultReceiverEpoch){ + this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset)), receiverEpoch); + } + else { + this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset))); + } + break; + } + } + + this.eventhubReceiver.setPrefetchCount(maxPrefetchCount); + this.eventhubReceiver.setReceiveTimeout(this.receiverTimeout); + logger.info("Successfully create eventhub receiver for partition {}, max_event_rate {}, max_prefetch_rate {}, receive_timeout {}, offset {}, ", + this.partitionId, + this.maxEventRate, + this.maxPrefetchCount, + this.receiverTimeout, + this.currentOffset); + } + + public Duration getReceiverTimeout() { + return receiverTimeout; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java new file mode 100644 index 0000000000000..c2c37bbb58c9b --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java @@ -0,0 +1,128 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.api.java.tuple.Tuple2; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * Created by jozh on 5/24/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + * Cause eventhub client can only access one partition at one time, so here we should have multiple eventhub clients + * In this worker thread, it will receive event from each partition in round robin mode, any partition failed to retrive + * events will lead thread exception, and leverage flink HA framework to start from begining again + */ +public class EventhubConsumerThread extends Thread { + private final Logger logger; + private final Handover handover; + private final Properties eventhubProps; + private final EventhubPartitionState[] subscribedPartitionStates; + private final Map clients; + private volatile boolean running; + + public EventhubConsumerThread( + Logger logger, + Handover handover, + Properties eventhubProps, + String threadName, + EventhubPartitionState[] subscribedPartitionStates) throws Exception{ + + super(threadName); + setDaemon(true); + + this.logger = logger; + this.handover = handover; + this.eventhubProps = eventhubProps; + this.subscribedPartitionStates = subscribedPartitionStates; + this.running = true; + + this.clients = new HashMap<>(this.subscribedPartitionStates.length); + for (int i = 0; i < this.subscribedPartitionStates.length; i++){ + EventhubClientWrapper client = new EventhubClientWrapper(); + this.clients.put(this.subscribedPartitionStates[i], client); + } + } + + public void shutdown(){ + logger.info("Shutdown eventhub consumer thread {} on demand", this.getName()); + running = false; + handover.wakeupProducer(); + } + + @Override + public void run() { + if (!running){ + logger.info("Eventhub consumer thread is set to STOP, thread {} exit", this.getName()); + return; + } + + try { + logger.info("Starting create {} eventhub clients on {}", this.subscribedPartitionStates.length, this.getName()); + for (Map.Entry client : clients.entrySet()){ + EventhubPartitionState state = client.getKey(); + client.getValue().createReveiver(this.eventhubProps, Integer.toString(state.getPartition().getParitionId()), state.getOffset()); + } + } + catch (Throwable t){ + logger.error("Create eventhub client of {}, error: {}", this.getName(), t); + handover.reportError(t); + clearReceiveClients(); + return; + } + + try { + int currentClientIndex = 0; + while (running){ + EventhubPartitionState partitionState = subscribedPartitionStates[currentClientIndex]; + EventhubClientWrapper client = clients.get(partitionState); + Iterable events = client.receive(); + if (events != null){ + handover.produce(Tuple2.of(partitionState.getPartition(), events)); + logger.debug("Received event from {} on {}", partitionState.getPartition().toString(), this.getName()); + } + else { + logger.warn("Receive events from {} timeout, timeout set to {}, thread {}", + partitionState.getPartition().toString(), + client.getReceiverTimeout(), + this.getName()); + } + + currentClientIndex++; + currentClientIndex = currentClientIndex % subscribedPartitionStates.length; + } + } + catch (Throwable t){ + logger.error("Receving events error, {}", t); + handover.reportError(t); + } + finally { + logger.info("Exit from eventhub consumer thread, {}", this.getName()); + handover.close(); + clearReceiveClients(); + } + + logger.info("EventhubConsumerThread {} quit", this.getName()); + } + + private void clearReceiveClients(){ + if (clients == null){ + return; + } + + for (Map.Entry client : clients.entrySet()){ + try { + client.getValue().close(); + logger.info("Eventhub client for partition {} closed", client.getKey().getPartition().getParitionId()); + } + catch (Throwable t){ + logger.warn("Error while close eventhub client for partition {}, error is {}", + client.getKey().getPartition().getParitionId(), + t.getMessage()); + } + } + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java new file mode 100644 index 0000000000000..950efde56b824 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java @@ -0,0 +1,12 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +/** + * Created by jozh on 5/22/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ +public enum EventhubOffsetType { + None, + PreviousCheckpoint, + InputByteOffset, + InputTimeOffset +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java new file mode 100644 index 0000000000000..f8ac4d6c5f7c8 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java @@ -0,0 +1,70 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; + +import java.io.Serializable; +import java.util.Properties; + +/** + * Created by jozh on 5/23/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ + +public class EventhubPartition implements Serializable { + private static final long serialVersionUID = 134878919919793479L; + private final int cachedHash; + private final String policyName; + private final String policyKey; + private final String namespace; + private final String name; + + public int getParitionId() { + return paritionId; + } + + public String getPartitionName(){ + return namespace + "-" + name; + } + + private final int paritionId; + + public EventhubPartition(Properties props, int parition){ + this(props.getProperty("eventhubs.policyname"), + props.getProperty("eventhubs.policykey"), + props.getProperty("eventhubs.namespace"), + props.getProperty("eventhubs.name"), + parition); + } + + public EventhubPartition(String policyName, String policyKey, String namespace, String name, int paritionId){ + Preconditions.checkArgument(paritionId >= 0); + + this.policyName = Preconditions.checkNotNull(policyName); + this.policyKey = Preconditions.checkNotNull(policyKey); + this.name = Preconditions.checkNotNull(name); + this.namespace = Preconditions.checkNotNull(namespace); + this.paritionId = paritionId; + this.cachedHash = 31 * (this.namespace + this.name).hashCode() + paritionId; + } + + @Override + public String toString() { + return "EventhubPartition, namespace: " + this.namespace + + " name: " + this.name + + " partition: " + this.paritionId; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof EventhubPartition){ + return this.hashCode() == ((EventhubPartition) obj).hashCode(); + } + + return false; + } + + @Override + public int hashCode() { + return this.cachedHash; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java new file mode 100644 index 0000000000000..3403ccac0d69e --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java @@ -0,0 +1,29 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +/** + * Created by jozh on 5/23/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ + +public class EventhubPartitionState { + private final EventhubPartition partition; + private volatile String offset; + + public EventhubPartitionState(EventhubPartition partition, String offset){ + this.partition = partition; + this.offset = offset; + } + + public final String getOffset() { + return this.offset; + } + + public final void setOffset(String offset) { + this.offset = offset; + } + + public EventhubPartition getPartition() { + return this.partition; + } +} + diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java new file mode 100644 index 0000000000000..2882703761baa --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java @@ -0,0 +1,38 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; + +/** + * Created by jozh on 6/16/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ + +public class EventhubPartitionStateWithPeriodicWatermarks extends EventhubPartitionState { + private final AssignerWithPeriodicWatermarks timestampsAndWatermarks; + private long partitionWatermark; + + public EventhubPartitionStateWithPeriodicWatermarks(EventhubPartition key, String value, AssignerWithPeriodicWatermarks timestampsAndWatermarks) { + super(key, value); + this.timestampsAndWatermarks = timestampsAndWatermarks; + this.partitionWatermark = Long.MIN_VALUE; + } + + public long getTimestampForRecord(T record, long kafkaEventTimestamp) { + return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp); + } + + public long getCurrentWatermarkTimestamp() { + Watermark wm = timestampsAndWatermarks.getCurrentWatermark(); + if (wm != null) { + partitionWatermark = Math.max(partitionWatermark, wm.getTimestamp()); + } + return partitionWatermark; + } + + @Override + public String toString() { + return "EventhubPartitionStateWithPeriodicWatermarks: partition=" + getPartition() + + ", offset=" + getOffset() + ", watermark=" + partitionWatermark; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java new file mode 100644 index 0000000000000..d68f8a00a077a --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java @@ -0,0 +1,48 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.watermark.Watermark; + +import javax.annotation.Nullable; + +/** + * Created by jozh on 6/16/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ + +public class EventhubPartitionStateWithPunctuatedWatermarks extends EventhubPartitionState { + private final AssignerWithPunctuatedWatermarks timestampsAndWatermarks; + private long partitionWatermark; + + public EventhubPartitionStateWithPunctuatedWatermarks(EventhubPartition key, String value, AssignerWithPunctuatedWatermarks timestampsAndWatermarks) { + super(key, value); + this.timestampsAndWatermarks = timestampsAndWatermarks; + this.partitionWatermark = Long.MIN_VALUE; + } + + public long getTimestampForRecord(T record, long kafkaEventTimestamp) { + return timestampsAndWatermarks.extractTimestamp(record, kafkaEventTimestamp); + } + + @Nullable + public Watermark checkAndGetNewWatermark(T record, long timestamp) { + Watermark mark = timestampsAndWatermarks.checkAndGetNextWatermark(record, timestamp); + if (mark != null && mark.getTimestamp() > partitionWatermark) { + partitionWatermark = mark.getTimestamp(); + return mark; + } + else { + return null; + } + } + + public long getCurrentPartitionWatermark() { + return partitionWatermark; + } + + @Override + public String toString() { + return "EventhubPartitionStateWithPunctuatedWatermarks: partition=" + getPartition() + + ", offset=" + getOffset() + ", watermark=" + partitionWatermark; + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java new file mode 100644 index 0000000000000..1cc1fd3dd2626 --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java @@ -0,0 +1,95 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.metrics.Counter; + +import com.microsoft.azure.eventhubs.EventData; +import com.microsoft.azure.eventhubs.EventHubClient; +import com.microsoft.azure.servicebus.ConnectionStringBuilder; +import com.microsoft.azure.servicebus.ServiceBusException; +import org.slf4j.Logger; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Properties; +/** + * Created by jozh on 6/20/2017. + */ + +public class EventhubProducerThread extends Thread { + private final Logger logger; + private final ProducerCache producerCache; + private final Properties eventhubProps; + private final EventHubClient producer; + private volatile boolean running; + private Counter commitSendCount; + + public EventhubProducerThread( + Logger logger, + String threadName, + ProducerCache producerCache, + Properties eventhubProps, + Counter commitSendCount) throws IOException, ServiceBusException{ + + super(threadName); + setDaemon(true); + + this.logger = logger; + this.producerCache = producerCache; + this.eventhubProps = eventhubProps; + this.commitSendCount = commitSendCount; + + ConnectionStringBuilder connectionStringBuilder = new ConnectionStringBuilder( + eventhubProps.getProperty("eventhubs.namespace"), + eventhubProps.getProperty("eventhubs.name"), + eventhubProps.getProperty("eventhubs.policyname"), + eventhubProps.getProperty("eventhubs.policykey")); + this.producer = EventHubClient.createFromConnectionStringSync(connectionStringBuilder.toString()); + this.running = true; + } + + public void shutdown(){ + logger.info("Shutdown eventhub producer thread {} on demand", this.getName()); + running = false; + } + + @Override + public void run() { + if (!running){ + logger.info("Eventhub producer thread is set to STOP, thread {} exit", this.getName()); + return; + } + + try { + logger.info("Eventhub producer thread {} started", this.getName()); + while (running){ + final ArrayList events = producerCache.pollNextBatch(); + if (events != null && events.size() > 0){ + producer.sendSync(events); + commitSendCount.inc(events.size()); + logger.info("Eventhub producer thread send {} events success", events.size()); + } + else { + logger.debug("Eventhub producer thread received a null eventdata from producer cache"); + } + } + } + catch (Throwable t){ + logger.error("Sending events error, {}", t.toString()); + producerCache.reportError(t); + } + finally { + logger.info("Exit from eventhub producer thread, {}", this.getName()); + if (producer != null){ + try { + producer.closeSync(); + } + catch (Exception ex) { + logger.error("Close eventhubclient {} error {}", eventhubProps.getProperty("eventhubs.name"), ex.getMessage()); + producerCache.reportError(ex); + } + } + } + + logger.info("EventhubProducerThread {} quit", this.getName()); + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java new file mode 100644 index 0000000000000..cc6098f078c3c --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java @@ -0,0 +1,108 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.ExceptionUtils; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.ThreadSafe; + +import java.io.Closeable; +import java.util.concurrent.ConcurrentLinkedQueue; + + +/** + * Created by jozh on 5/23/2017. + * Flink eventhub connnector has implemented with same design of flink kafka connector + */ +@ThreadSafe +public final class Handover implements Closeable { + private static final int MAX_EVENTS_BLOCK_IN_QUEUE = 1000; + private static final Logger logger = LoggerFactory.getLogger(Handover.class); + private ConcurrentLinkedQueue>> eventQueue = new ConcurrentLinkedQueue(); + + private volatile boolean allProducerWakeup = true; + + private Throwable error; + + public Tuple2> pollNext() throws Exception{ + logger.debug("###Begin to poll data from event cache queue"); + synchronized (eventQueue){ + while (eventQueue.isEmpty() && error == null){ + logger.debug("### No data in the msg queue, waiting... "); + eventQueue.wait(); + } + + logger.debug("### Get notified from consummer thread"); + Tuple2> events = eventQueue.poll(); + if (events != null && events.f0 != null && events.f1 != null){ + logger.debug("### Get event data from {}", events.f0.toString()); + int queueSize = eventQueue.size(); + if (queueSize < MAX_EVENTS_BLOCK_IN_QUEUE / 2){ + eventQueue.notifyAll(); + } + return events; + } + else { + ExceptionUtils.rethrowException(error, error.getMessage()); + return null; + } + } + } + + public void produce(final Tuple2> events) throws InterruptedException{ + if (events == null || events.f0 == null || events.f1 == null){ + logger.error("Received empty events from event producer"); + return; + } + + synchronized (eventQueue){ + while (eventQueue.size() > MAX_EVENTS_BLOCK_IN_QUEUE){ + logger.warn("Event queue is full, current size is {}", eventQueue.size()); + eventQueue.wait(); + } + + eventQueue.add(events); + eventQueue.notifyAll(); + logger.debug("Add received events into queue"); + } + } + + @Override + public void close() { + synchronized (eventQueue){ + logger.info("Close handover on demand"); + eventQueue.clear(); + if (error == null){ + error = new Throwable("Handover closed on command"); + } + + eventQueue.notifyAll(); + } + } + + public void reportError(Throwable t) { + if (t == null){ + return; + } + + synchronized (eventQueue){ + if (error == null){ + error = t; + } + eventQueue.clear(); + eventQueue.notifyAll(); + logger.info("Consumer thread report a error: {}", error.getMessage()); + } + } + + public void wakeupProducer() { + synchronized (eventQueue){ + logger.info("Wakeup producer on demand"); + eventQueue.clear(); + eventQueue.notifyAll(); + } + } +} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java new file mode 100644 index 0000000000000..35d8e300c487f --- /dev/null +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java @@ -0,0 +1,124 @@ +package org.apache.flink.streaming.connectors.eventhubs.internals; + +import org.apache.flink.util.ExceptionUtils; + +import com.microsoft.azure.eventhubs.EventData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Date; +import java.util.concurrent.ArrayBlockingQueue; + +/** + * Created by jozh on 6/20/2017. + */ +public final class ProducerCache implements Closeable, Serializable { + private static final Logger logger = LoggerFactory.getLogger(ProducerCache.class); + private static final long defaultCheckQueueStatusInterval = 50; + public static final int DEFAULTCAPACITY = 100; + public static final long DEFAULTTIMEOUTMILLISECOND = 100; + private final ArrayBlockingQueue cacheQueue; + private final int queueCapacity; + private final long pollTimeout; + private Date lastPollTime; + private Throwable error; + private volatile boolean closed; + + public ProducerCache(){ + this(DEFAULTCAPACITY, DEFAULTTIMEOUTMILLISECOND); + } + + public ProducerCache(int capacity){ + this(capacity, DEFAULTTIMEOUTMILLISECOND); + } + + public ProducerCache(int capacity, long timeout){ + this.queueCapacity = capacity; + this.pollTimeout = timeout; + this.cacheQueue = new ArrayBlockingQueue(this.queueCapacity); + this.lastPollTime = new Date(); + this.closed = false; + } + + public void put(EventData value) throws InterruptedException{ + if (value == null){ + logger.error("Received empty events from event producer"); + return; + } + + synchronized (cacheQueue){ + while (cacheQueue.remainingCapacity() <= 0 && !closed){ + logger.warn("Event queue is full, current size is {}", cacheQueue.size()); + cacheQueue.wait(); + } + + if (closed){ + logger.info("Cache is closed, event is dropped."); + return; + } + + cacheQueue.add(value); + cacheQueue.notifyAll(); + + logger.debug("Add event into queue"); + } + } + + public ArrayList pollNextBatch() throws InterruptedException{ + logger.debug("###Begin to poll all data from event cache queue"); + + synchronized (cacheQueue){ + while (!isPollTimeout() && !closed && cacheQueue.remainingCapacity() > 0){ + cacheQueue.wait(defaultCheckQueueStatusInterval); + } + + final ArrayList result = new ArrayList<>(cacheQueue.size()); + for (EventData item : cacheQueue){ + result.add(item); + } + cacheQueue.clear(); + cacheQueue.notifyAll(); + + lastPollTime = new Date(); + return result; + } + } + + public void reportError(Throwable t) { + if (t == null){ + return; + } + + synchronized (cacheQueue){ + if (error == null){ + error = t; + } + logger.info("Producer thread report a error: {}", t.toString()); + } + } + + @Override + public void close() { + synchronized (cacheQueue){ + logger.info("Close cache on demand"); + closed = true; + cacheQueue.notifyAll(); + } + } + + public void checkErr() throws Exception { + synchronized (cacheQueue){ + if (error != null){ + ExceptionUtils.rethrowException(error, error.getMessage()); + } + } + } + + private boolean isPollTimeout(){ + long pollInterval = (new Date()).getTime() - lastPollTime.getTime(); + return pollInterval > pollTimeout; + } +} diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index bc3f82f686c44..2ed3b7974098b 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -54,6 +54,7 @@ under the License. flink-connector-nifi flink-connector-cassandra flink-connector-filesystem + flink-connector-eventhubs + diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java index ba4abd7bfdd5d..c04a2cd055bd3 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs; import org.apache.flink.api.common.state.ListState; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java index 61db348869c5f..344e1f1a5ad59 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubProducer.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs; import org.apache.flink.configuration.Configuration; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java index cae6327052930..3212702da1be1 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventFetcher.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.api.java.tuple.Tuple2; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java index 9470482ccf5d0..e00c97750f16e 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.util.Preconditions; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java~HEAD b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java~HEAD index 9470482ccf5d0..e00c97750f16e 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java~HEAD +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java~HEAD @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.util.Preconditions; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java index c2c37bbb58c9b..7ab0efa2c8134 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.api.java.tuple.Tuple2; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java index 950efde56b824..4a97a95ff8546 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; /** diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java index f8ac4d6c5f7c8..1eb33a28ca06e 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java index 3403ccac0d69e..d1547a539bde6 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionState.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; /** diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java index 2882703761baa..1fab7ff074633 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPeriodicWatermarks.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java index d68f8a00a077a..ae3d07ea3cd91 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartitionStateWithPunctuatedWatermarks.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java index 1cc1fd3dd2626..b51236749386f 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubProducerThread.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.metrics.Counter; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java index cc6098f078c3c..277b4bb7a1380 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/Handover.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.api.java.tuple.Tuple2; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java index 35d8e300c487f..3c49209e3d44f 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java @@ -1,3 +1,21 @@ +/* + * 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.eventhubs.internals; import org.apache.flink.util.ExceptionUtils; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/ReadFromEventhub.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/ReadFromEventhub.java index 8b39088171a0d..d04e921669b5b 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/ReadFromEventhub.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/ReadFromEventhub.java @@ -1,3 +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. + */ + package org.apache.flink.streaming.examples.eventhub; import org.apache.flink.api.common.restartstrategy.RestartStrategies; diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/WriteToEventhub.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/WriteToEventhub.java index 28803e2628f7c..397daf9f452ac 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/WriteToEventhub.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/eventhub/WriteToEventhub.java @@ -1,3 +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. + */ + package org.apache.flink.streaming.examples.eventhub; import org.apache.flink.api.common.restartstrategy.RestartStrategies; From ca8744d70e1bb7ee365b72bb24dd3dc776d9080c Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Thu, 14 Sep 2017 15:57:25 +0800 Subject: [PATCH 121/129] Modify D:\Tools\jdk1.8.0_101 back to $JAVA_HOME, since it is not running on windows platform --- .../apache/flink/runtime/clusterframework/BootstrapTools.java | 4 ++-- .../org/apache/flink/yarn/AbstractYarnClusterDescriptor.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index 7e5425f60257a..7b7c189910ceb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -371,8 +371,8 @@ public static String getTaskManagerShellCommand( Class mainClass) { final Map startCommandValues = new HashMap<>(); - startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster - //startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster + //startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster + startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster ArrayList params = new ArrayList<>(); params.add(String.format("-Xms%dm", tmParams.taskManagerHeapSizeMB())); 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 8f0ac0356e148..cec4dcce67efc 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 @@ -1352,8 +1352,8 @@ protected ContainerLaunchContext setupApplicationMasterContainer( ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); final Map startCommandValues = new HashMap<>(); - startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster - //startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster + //startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster + startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster startCommandValues.put("jvmmem", "-Xmx" + Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration) + "m"); From 0a76d470835e418c1523cfc9793254562af823ed Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Thu, 14 Sep 2017 15:57:25 +0800 Subject: [PATCH 122/129] Modify %JAVA_HOME% to $JAVA_HOME, since it is not running on windows platform --- .../apache/flink/runtime/clusterframework/BootstrapTools.java | 4 ++-- .../org/apache/flink/yarn/AbstractYarnClusterDescriptor.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index 7e5425f60257a..7b7c189910ceb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -371,8 +371,8 @@ public static String getTaskManagerShellCommand( Class mainClass) { final Map startCommandValues = new HashMap<>(); - startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster - //startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster + //startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster + startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster ArrayList params = new ArrayList<>(); params.add(String.format("-Xms%dm", tmParams.taskManagerHeapSizeMB())); 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 8f0ac0356e148..cec4dcce67efc 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 @@ -1352,8 +1352,8 @@ protected ContainerLaunchContext setupApplicationMasterContainer( ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); final Map startCommandValues = new HashMap<>(); - startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster - //startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster + //startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster + startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster startCommandValues.put("jvmmem", "-Xmx" + Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration) + "m"); From ecd8015fe6302a02949520d81114d92fccf5d748 Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Thu, 14 Sep 2017 16:45:14 +0800 Subject: [PATCH 123/129] [FLINK-7474]Fix checkstyle error --- .../eventhubs/FlinkEventHubConsumer.java | 2 +- .../internals/EventhubClientWrapper.java~HEAD | 218 ------------------ .../internals/EventhubConsumerThread.java | 2 +- .../internals/EventhubOffsetType.java | 2 +- 4 files changed, 3 insertions(+), 221 deletions(-) delete mode 100644 flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java~HEAD diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java index c04a2cd055bd3..261b2e51f2a51 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.flink.streaming.connectors.eventhubs; import org.apache.flink.api.common.state.ListState; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java~HEAD b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java~HEAD deleted file mode 100644 index e00c97750f16e..0000000000000 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java~HEAD +++ /dev/null @@ -1,218 +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.eventhubs.internals; - -import org.apache.flink.util.Preconditions; - -import com.microsoft.azure.eventhubs.EventData; -import com.microsoft.azure.eventhubs.EventHubClient; -import com.microsoft.azure.eventhubs.PartitionReceiver; -import com.microsoft.azure.servicebus.ConnectionStringBuilder; -import com.microsoft.azure.servicebus.ServiceBusException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.Serializable; -import java.net.URI; -import java.net.URISyntaxException; -import java.time.Duration; -import java.time.Instant; -import java.util.Properties; -import java.util.concurrent.ExecutionException; - -/** - * Created by jozh on 6/14/2017. - * Flink eventhub connnector has implemented with same design of flink kafka connector - */ -public class EventhubClientWrapper implements Serializable { - private static final long serialVersionUID = -5319150387753930840L; - private static final Logger logger = LoggerFactory.getLogger(EventhubClientWrapper.class); - private EventHubClient eventHubClient; - private PartitionReceiver eventhubReceiver; - private ConnectionStringBuilder connectionString; - private String consumerGroup; - private Long receiverEpoch; - - private Duration receiverTimeout; - private EventhubOffsetType offsetType; - private String currentOffset; - private String partitionId; - - private final int minPrefetchCount = 10; - private int maxPrefetchCount = 999; - private int maxEventRate = 0; - private final Long defaultReceiverEpoch = -1L; - private final String defaultReceiverTimeout = "60000"; - - public void createReveiver(Properties eventhubParams, String partitionId) - throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ - int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10")); - this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM); - } - - public void createReveiver(Properties eventhubParams, String partitionId, String offset) - throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ - int maxEventRate = Integer.parseInt(eventhubParams.getProperty("eventhubs.maxRate", "10")); - this.createReveiver(eventhubParams, partitionId, maxEventRate, offset); - } - - public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate) - throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ - this.createReveiver(eventhubParams, partitionId, maxEventRate, PartitionReceiver.START_OF_STREAM); - } - - /*Will not implement a standalone offset store here, will leverage flink state to save the offset of eventhub*/ - public void createReveiver(Properties eventhubParams, String partitionId, int maxEventRate, String offset) - throws IllegalArgumentException, URISyntaxException, IOException, ServiceBusException{ - if (eventhubParams.containsKey("eventhubs.uri") && eventhubParams.containsKey("eventhubs.namespace")) { - throw new IllegalArgumentException("Eventhubs URI and namespace cannot both be specified at the same time."); - } - - if (eventhubParams.containsKey("eventhubs.namespace")){ - this.connectionString = new ConnectionStringBuilder( - eventhubParams.getProperty("eventhubs.namespace"), - eventhubParams.getProperty("eventhubs.name"), - eventhubParams.getProperty("eventhubs.policyname"), - eventhubParams.getProperty("eventhubs.policykey")); - } - else if (eventhubParams.containsKey("eventhubs.uri")){ - this.connectionString = new ConnectionStringBuilder(new URI( - eventhubParams.getProperty("eventhubs.uri")), - eventhubParams.getProperty("eventhubs.name"), - eventhubParams.getProperty("eventhubs.policyname"), - eventhubParams.getProperty("eventhubs.policykey")); - } - else { - throw new IllegalArgumentException("Either Eventhubs URI or namespace nust be specified."); - } - - this.partitionId = Preconditions.checkNotNull(partitionId, "partitionId is no valid, cannot be null or empty"); - this.consumerGroup = eventhubParams.getProperty("eventhubs.consumergroup", EventHubClient.DEFAULT_CONSUMER_GROUP_NAME); - this.receiverEpoch = Long.parseLong(eventhubParams.getProperty("eventhubs.epoch", defaultReceiverEpoch.toString())); - this.receiverTimeout = Duration.ofMillis(Long.parseLong(eventhubParams.getProperty("eventhubs.receiver.timeout", defaultReceiverTimeout))); - this.offsetType = EventhubOffsetType.None; - this.currentOffset = PartitionReceiver.START_OF_STREAM; - - String previousOffset = offset; - - if (previousOffset.compareTo(PartitionReceiver.START_OF_STREAM) != 0 && previousOffset != null) { - - offsetType = EventhubOffsetType.PreviousCheckpoint; - currentOffset = previousOffset; - - } else if (eventhubParams.containsKey("eventhubs.filter.offset")) { - - offsetType = EventhubOffsetType.InputByteOffset; - currentOffset = eventhubParams.getProperty("eventhubs.filter.offset"); - - } else if (eventhubParams.containsKey("eventhubs.filter.enqueuetime")) { - - offsetType = EventhubOffsetType.InputTimeOffset; - currentOffset = eventhubParams.getProperty("eventhubs.filter.enqueuetime"); - } - - this.maxEventRate = maxEventRate; - - if (maxEventRate > 0 && maxEventRate < minPrefetchCount) { - maxPrefetchCount = minPrefetchCount; - } - else if (maxEventRate >= minPrefetchCount && maxEventRate < maxPrefetchCount) { - maxPrefetchCount = maxEventRate + 1; - } - else { - maxEventRate = maxPrefetchCount - 1; - } - - this.createReceiverInternal(); - } - - public Iterable receive () throws ExecutionException, InterruptedException { - return this.eventhubReceiver.receive(maxEventRate).get(); - } - - public void close(){ - logger.info("Close eventhub client on demand of partition {}", this.partitionId); - if (this.eventhubReceiver != null){ - try { - this.eventhubReceiver.closeSync(); - } - catch (ServiceBusException ex){ - logger.error("Close eventhub client of partition {} failed, reason: {}", this.partitionId, ex.getMessage()); - } - } - } - - private void createReceiverInternal() throws IOException, ServiceBusException{ - this.eventHubClient = EventHubClient.createFromConnectionStringSync(this.connectionString.toString()); - - switch (this.offsetType){ - case None: { - if (this.receiverEpoch > defaultReceiverEpoch){ - this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, receiverEpoch); - } - else { - this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); - } - break; - } - case PreviousCheckpoint: { - if (this.receiverEpoch > defaultReceiverEpoch){ - this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch); - } - else { - this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); - } - break; - } - case InputByteOffset: { - if (this.receiverEpoch > defaultReceiverEpoch){ - this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, currentOffset, false, receiverEpoch); - } - else { - this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, currentOffset, false); - } - break; - } - case InputTimeOffset: { - if (this.receiverEpoch > defaultReceiverEpoch){ - this.eventhubReceiver = this.eventHubClient.createEpochReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset)), receiverEpoch); - } - else { - this.eventhubReceiver = this.eventHubClient.createReceiverSync(consumerGroup, partitionId, Instant.ofEpochSecond(Long.parseLong(currentOffset))); - } - break; - } - } - - this.eventhubReceiver.setPrefetchCount(maxPrefetchCount); - this.eventhubReceiver.setReceiveTimeout(this.receiverTimeout); - logger.info("Successfully create eventhub receiver for partition {}, max_event_rate {}, max_prefetch_rate {}, receive_timeout {}, offset {}, ", - this.partitionId, - this.maxEventRate, - this.maxPrefetchCount, - this.receiverTimeout, - this.currentOffset); - } - - public Duration getReceiverTimeout() { - return receiverTimeout; - } -} diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java index 7ab0efa2c8134..a0f9f261d512e 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubConsumerThread.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.flink.streaming.connectors.eventhubs.internals; import org.apache.flink.api.java.tuple.Tuple2; diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java index 4a97a95ff8546..e745ded06afeb 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubOffsetType.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.flink.streaming.connectors.eventhubs.internals; /** From 3d3b8fb550afc56bd08dbb6d9f865ebf6eab7f39 Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Fri, 15 Sep 2017 11:48:23 +0800 Subject: [PATCH 124/129] [FLINK-7474]Fix check build error, deprecated CheckpointedRestoring --- .../streaming/connectors/eventhubs/FlinkEventHubConsumer.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java index 261b2e51f2a51..c09bcfba845be 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.checkpoint.CheckpointedRestoring; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; @@ -267,7 +266,7 @@ else if (this.restoreToOffset.isEmpty()){ } } - @Override + //deprecated for CheckpointedRestoring public void restoreState(HashMap eventhubPartitionOffsets) throws Exception { LOGGER.info("{} (taskIdx={}) restoring offsets from an older version.", getClass().getSimpleName(), getRuntimeContext().getIndexOfThisSubtask()); From 5e42a75ca812c02c2fc3b2ffe75efb7d6a7d4fb1 Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Fri, 15 Sep 2017 12:58:02 +0800 Subject: [PATCH 125/129] [FLINK-7474]Fix check build error, deprecated CheckpointedRestoring --- .../streaming/connectors/eventhubs/FlinkEventHubConsumer.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java index c09bcfba845be..e5598597de73f 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/FlinkEventHubConsumer.java @@ -60,8 +60,7 @@ public class FlinkEventHubConsumer extends RichParallelSourceFunction implements CheckpointedFunction, - ResultTypeQueryable, - CheckpointedRestoring> { + ResultTypeQueryable { private static final long serialVersionUID = -3247976470793561346L; protected static final Logger LOGGER = LoggerFactory.getLogger(FlinkEventHubConsumer.class); protected static final String DEFAULTOFFSETSTATENAME = "flink.eventhub.offset"; From 6dc4c8008059bf2c6e1f59c594dc72df5533af7c Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Fri, 15 Sep 2017 14:03:48 +0800 Subject: [PATCH 126/129] [FLINK-7474]Fix bug, maxEventRate is not set right if user set it over 999 --- .../connectors/eventhubs/internals/EventhubClientWrapper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java index e00c97750f16e..39f22a263a9e7 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubClientWrapper.java @@ -138,7 +138,7 @@ else if (maxEventRate >= minPrefetchCount && maxEventRate < maxPrefetchCount) { maxPrefetchCount = maxEventRate + 1; } else { - maxEventRate = maxPrefetchCount - 1; + this.maxEventRate = maxPrefetchCount - 1; } this.createReceiverInternal(); From ca11bb6db260e345ceb9465bd39f487142b92234 Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Fri, 15 Sep 2017 16:06:26 +0800 Subject: [PATCH 127/129] [FLINK-7474]Fix checkstyle error --- .../runtime/clusterframework/BootstrapTools.java | 8 ++++++-- .../flink/yarn/AbstractYarnClusterDescriptor.java | 11 ++++++++--- .../flink/yarn/YarnApplicationMasterRunner.java | 9 ++++----- 3 files changed, 18 insertions(+), 10 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java index 7b7c189910ceb..b86054f5ca7d9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/BootstrapTools.java @@ -371,8 +371,12 @@ public static String getTaskManagerShellCommand( Class mainClass) { final Map startCommandValues = new HashMap<>(); - //startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster - startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster + if (System.getProperty("os.name").toLowerCase().startsWith("windows")){ + startCommandValues.put("java", "%JAVA_HOME%/bin/java"); + } + else { + startCommandValues.put("java", "$JAVA_HOME/bin/java"); + } ArrayList params = new ArrayList<>(); params.add(String.format("-Xms%dm", tmParams.taskManagerHeapSizeMB())); 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 cec4dcce67efc..0eb2cc5c4cf23 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 @@ -254,7 +254,7 @@ private void isReadyForDeployment(ClusterSpecification clusterSpecification) thr // If not configured, it is set to the number of task slots int numYarnVcores = conf.getInt(YarnConfiguration.NM_VCORES, YarnConfiguration.DEFAULT_NM_VCORES); numYarnVcores = numYarnVcores <= 0 ? YarnConfiguration.DEFAULT_NM_VCORES : numYarnVcores; - int configuredVcores = flinkConfiguration.getInteger(ConfigConstants.YARN_VCORES, slots); + int configuredVcores = flinkConfiguration.getInteger(YarnConfigOptions.VCORES, clusterSpecification.getSlotsPerTaskManager()); // don't configure more than the maximum configured number of vcores if (configuredVcores > numYarnVcores) { throw new IllegalConfigurationException( @@ -1352,8 +1352,13 @@ protected ContainerLaunchContext setupApplicationMasterContainer( ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); final Map startCommandValues = new HashMap<>(); - //startCommandValues.put("java", "%JAVA_HOME%/bin/java");//For AP yarn cluster - startCommandValues.put("java", "$JAVA_HOME/bin/java");//For HDI yarn cluster + if (System.getProperty("os.name").toLowerCase().startsWith("windows")){ + startCommandValues.put("java", "%JAVA_HOME%/bin/java"); + } + else { + startCommandValues.put("java", "$JAVA_HOME/bin/java"); + } + startCommandValues.put("jvmmem", "-Xmx" + Utils.calculateHeapSize(jobManagerMemoryMb, flinkConfiguration) + "m"); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java index 2c2abcec208aa..e71644e953674 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java @@ -94,7 +94,7 @@ public class YarnApplicationMasterRunner { private static final FiniteDuration TASKMANAGER_REGISTRATION_TIMEOUT = new FiniteDuration(5, TimeUnit.MINUTES); /** The process environment variables. */ - private static final Map ENV = getSystemEnv();//System.getenv(); + private static final Map ENV = getSystemEnv(); /** The exit code returned if the initialization of the application master failed. */ private static final int INIT_ERROR_EXIT_CODE = 31; @@ -102,20 +102,19 @@ public class YarnApplicationMasterRunner { /** The exit code returned if the process exits because a critical actor died. */ private static final int ACTOR_DIED_EXIT_CODE = 32; - /** - * Add this private static method to convert the hostname to lowercase + * Add this private static method to convert the hostname to lowercase. */ private static Map getSystemEnv(){ final Map origSysEnv = System.getenv(); final Map modifiedEnv = new HashMap<>(); - for(Map.Entry entry : origSysEnv.entrySet()){ + for (Map.Entry entry : origSysEnv.entrySet()){ modifiedEnv.put(entry.getKey(), entry.getValue()); } String hostName = modifiedEnv.get(Environment.NM_HOST.key()); - if(hostName != null){ + if (hostName != null){ modifiedEnv.put(Environment.NM_HOST.key(), hostName.toLowerCase()); } From 2e3657cd50d64e5158f08bc8ac7395002f682bbb Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Mon, 25 Sep 2017 16:25:53 +0800 Subject: [PATCH 128/129] [FLINK-7474] Fix bug that flinkeventhubproducer will be in abnormal status and waiting forevet if eventhubproducerthread quit --- .../connectors/eventhubs/internals/ProducerCache.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java index 3c49209e3d44f..5b29c4ff95f71 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/ProducerCache.java @@ -61,7 +61,7 @@ public ProducerCache(int capacity, long timeout){ this.closed = false; } - public void put(EventData value) throws InterruptedException{ + public void put(EventData value) throws Exception{ if (value == null){ logger.error("Received empty events from event producer"); return; @@ -69,8 +69,9 @@ public void put(EventData value) throws InterruptedException{ synchronized (cacheQueue){ while (cacheQueue.remainingCapacity() <= 0 && !closed){ + checkErr(); logger.warn("Event queue is full, current size is {}", cacheQueue.size()); - cacheQueue.wait(); + cacheQueue.wait(defaultCheckQueueStatusInterval); } if (closed){ From 683abaf9daf154acc4b0a070eb05f1c5e115ef51 Mon Sep 17 00:00:00 2001 From: zhuganghuaonnet Date: Mon, 6 Nov 2017 18:57:09 +0800 Subject: [PATCH 129/129] Change Preconditions reference from hadoop back to flink bits --- .../connectors/eventhubs/internals/EventhubPartition.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java index 1eb33a28ca06e..9ae168ccf664c 100644 --- a/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java +++ b/flink-connectors/flink-connector-eventhubs/src/main/java/org/apache/flink/streaming/connectors/eventhubs/internals/EventhubPartition.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.connectors.eventhubs.internals; -import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; +import org.apache.flink.util.Preconditions; import java.io.Serializable; import java.util.Properties;